You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-dev@axis.apache.org by di...@apache.org on 2007/04/06 04:12:25 UTC

svn commit: r526026 [1/2] - in /webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http: ./ server/

Author: dims
Date: Thu Apr  5 19:12:24 2007
New Revision: 526026

URL: http://svn.apache.org/viewvc?view=rev&rev=526026
Log:
Fix for AXIS2-2449 - Request and response content streaming for SimpleHttpServer

Added:
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpConnection.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpConnectionImpl.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpRequest.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpRequestImpl.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpResponse.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpResponseImpl.java
Modified:
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/HTTPWorker.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpService.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultConnectionListener.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultHttpConnectionFactory.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultHttpConnectionManager.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpConnectionFactory.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpConnectionManager.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpFactory.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpServiceProcessor.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpUtils.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/LoggingProcessorDecorator.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/OutputBuffer.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/SimpleHttpServer.java
    webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/Worker.java

Modified: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/HTTPWorker.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/HTTPWorker.java?view=diff&rev=526026&r1=526025&r2=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/HTTPWorker.java (original)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/HTTPWorker.java Thu Apr  5 19:12:24 2007
@@ -16,10 +16,14 @@
 
 package org.apache.axis2.transport.http;
 
-import edu.emory.mathcs.backport.java.util.concurrent.CountDownLatch;
-import org.apache.axis2.AxisFault;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+
 import org.apache.axis2.Constants;
-import org.apache.axis2.addressing.EndpointReference;
 import org.apache.axis2.context.ConfigurationContext;
 import org.apache.axis2.context.MessageContext;
 import org.apache.axis2.context.OperationContext;
@@ -27,34 +31,28 @@
 import org.apache.axis2.description.AxisService;
 import org.apache.axis2.engine.Handler.InvocationResponse;
 import org.apache.axis2.transport.RequestResponseTransport;
+import org.apache.axis2.transport.http.server.AxisHttpRequest;
+import org.apache.axis2.transport.http.server.AxisHttpResponse;
 import org.apache.axis2.transport.http.server.HttpUtils;
-import org.apache.axis2.transport.http.server.OutputBuffer;
 import org.apache.axis2.transport.http.server.Worker;
 import org.apache.axis2.transport.http.util.RESTUtil;
-import org.apache.http.*;
-import org.apache.http.entity.ContentProducer;
-import org.apache.http.entity.EntityTemplate;
-import org.apache.http.entity.StringEntity;
+import org.apache.http.Header;
+import org.apache.http.HttpException;
+import org.apache.http.HttpStatus;
+import org.apache.http.MethodNotSupportedException;
 import org.apache.http.message.BasicHeader;
-import org.apache.http.message.BasicStatusLine;
+import org.apache.http.protocol.HTTP;
+import org.apache.http.util.EncodingUtils;
 import org.apache.ws.commons.schema.XmlSchema;
 
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.URI;
-import java.util.HashMap;
-import java.util.Map;
-
 public class HTTPWorker implements Worker {
 
     public HTTPWorker() {
     }
 
     public void service(
-            final HttpRequest request,
-            final HttpResponse response,
+            final AxisHttpRequest request,
+            final AxisHttpResponse response,
             final MessageContext msgContext) throws HttpException, IOException {
 
         ConfigurationContext configurationContext = msgContext.getConfigurationContext();
@@ -62,41 +60,40 @@
         final String contextPath =
                 (servicePath.startsWith("/") ? servicePath : "/" + servicePath) + "/";
 
-        HttpVersion ver = request.getRequestLine().getHttpVersion();
-        String uri = request.getRequestLine().getUri();
-        String method = request.getRequestLine().getMethod();
+        String uri = request.getRequestURI();
+        String method = request.getMethod();
         String soapAction = HttpUtils.getSoapAction(request);
-        OutputBuffer outbuffer;
         InvocationResponse pi;
 
         if (method.equals(HTTPConstants.HEADER_GET)) {
             if (uri.equals("/favicon.ico")) {
-                response.setStatusLine(new BasicStatusLine(ver, 301, "Redirect"));
+                response.setStatus(HttpStatus.SC_MOVED_PERMANENTLY);
                 response.addHeader(new BasicHeader("Location", "http://ws.apache.org/favicon.ico"));
                 return;
             }
             if (!uri.startsWith(contextPath)) {
-                response.setStatusLine(new BasicStatusLine(ver, 301, "Redirect"));
+                response.setStatus(HttpStatus.SC_MOVED_PERMANENTLY);
                 response.addHeader(new BasicHeader("Location", contextPath));
                 return;
             }
             if (uri.endsWith("axis2/services/")) {
-                response.setStatusLine(new BasicStatusLine(ver, 200, "OK"));
                 String s = HTTPTransportReceiver.getServicesHTML(configurationContext);
-                StringEntity entity = new StringEntity(s);
-                entity.setContentType("text/html");
-                response.setEntity(entity);
+                response.setStatus(HttpStatus.SC_OK);
+                response.setContentType("text/html");
+                OutputStream out = response.getOutputStream();
+                out.write(EncodingUtils.getBytes(s, HTTP.ISO_8859_1));
                 return;
             }
             if (uri.indexOf("?") < 0) {
                 if (!uri.endsWith(contextPath)) {
                     String serviceName = uri.replaceAll(contextPath, "");
                     if (serviceName.indexOf("/") < 0) {
-                        String res = HTTPTransportReceiver
+                        String s = HTTPTransportReceiver
                                 .printServiceHTML(serviceName, configurationContext);
-                        StringEntity entity = new StringEntity(res);
-                        entity.setContentType("text/html");
-                        response.setEntity(entity);
+                        response.setStatus(HttpStatus.SC_OK);
+                        response.setContentType("text/html");
+                        OutputStream out = response.getOutputStream();
+                        out.write(EncodingUtils.getBytes(s, HTTP.ISO_8859_1));
                         return;
                     }
                 }
@@ -104,53 +101,35 @@
             if (uri.endsWith("?wsdl2")) {
                 String serviceName = uri.substring(uri.lastIndexOf("/") + 1, uri.length() - 6);
                 HashMap services = configurationContext.getAxisConfiguration().getServices();
-                final AxisService service = (AxisService) services.get(serviceName);
+                AxisService service = (AxisService) services.get(serviceName);
                 if (service != null) {
-                    final String ip = getHostAddress(request);
-                    EntityTemplate entity = new EntityTemplate(new ContentProducer() {
-
-                        public void writeTo(final OutputStream outstream) throws IOException {
-                            service.printWSDL2(outstream, ip, servicePath);
-                        }
-
-                    });
-                    entity.setContentType("text/xml");
-                    response.setEntity(entity);
+                    String ip = getHostAddress(request);
+                    response.setStatus(HttpStatus.SC_OK);
+                    response.setContentType("text/xml");
+                    service.printWSDL2(response.getOutputStream(), ip, servicePath);
                     return;
                 }
             }
             if (uri.endsWith("?wsdl")) {
                 String serviceName = uri.substring(uri.lastIndexOf("/") + 1, uri.length() - 5);
                 HashMap services = configurationContext.getAxisConfiguration().getServices();
-                final AxisService service = (AxisService) services.get(serviceName);
+                AxisService service = (AxisService) services.get(serviceName);
                 if (service != null) {
-                    final String ip = getHostAddress(request);
-                    EntityTemplate entity = new EntityTemplate(new ContentProducer() {
-
-                        public void writeTo(final OutputStream outstream) throws IOException {
-                            service.printWSDL(outstream, ip, servicePath);
-                        }
-
-                    });
-                    entity.setContentType("text/xml");
-                    response.setEntity(entity);
+                    String ip = getHostAddress(request);
+                    response.setStatus(HttpStatus.SC_OK);
+                    response.setContentType("text/xml");
+                    service.printWSDL(response.getOutputStream(), ip, servicePath);
                     return;
                 }
             }
             if (uri.endsWith("?xsd")) {
                 String serviceName = uri.substring(uri.lastIndexOf("/") + 1, uri.length() - 4);
                 HashMap services = configurationContext.getAxisConfiguration().getServices();
-                final AxisService service = (AxisService) services.get(serviceName);
+                AxisService service = (AxisService) services.get(serviceName);
                 if (service != null) {
-                    EntityTemplate entity = new EntityTemplate(new ContentProducer() {
-
-                        public void writeTo(final OutputStream outstream) throws IOException {
-                            service.printSchema(outstream);
-                        }
-
-                    });
-                    entity.setContentType("text/xml");
-                    response.setEntity(entity);
+                    response.setStatus(HttpStatus.SC_OK);
+                    response.setContentType("text/xml");
+                    service.printSchema(response.getOutputStream());
                     return;
                 }
             }
@@ -167,58 +146,41 @@
                     service.populateSchemaMappings();
                     //write out the correct schema
                     Map schemaTable = service.getSchemaMappingTable();
-                    final XmlSchema schema = (XmlSchema) schemaTable.get(schemaName);
+                    XmlSchema schema = (XmlSchema) schemaTable.get(schemaName);
                     //schema found - write it to the stream
                     if (schema != null) {
-                        EntityTemplate entity = new EntityTemplate(new ContentProducer() {
-
-                            public void writeTo(final OutputStream outstream) {
-                                schema.write(outstream);
-                            }
-
-                        });
-                        entity.setContentType("text/xml");
-                        response.setEntity(entity);
+                        response.setStatus(HttpStatus.SC_OK);
+                        response.setContentType("text/xml");
+                        schema.write(response.getOutputStream());
                         return;
                     } else {
-                      final  InputStream in = service.getClassLoader()
-                                .getResourceAsStream(DeploymentConstants.META_INF + "/" + schemaName);
-                        if (in != null) {
-                            EntityTemplate entity = new EntityTemplate(new ContentProducer() {
-
-                                public void writeTo(final OutputStream outstream) {
-                                    try {
-                                        boolean checkLength = true;
-                                        int length = Integer.MAX_VALUE;
-                                        int nextValue = in.read();
-                                        if (checkLength) length--;
-                                        while (-1 != nextValue && length >= 0) {
-                                            outstream.write(nextValue);
-                                            nextValue = in.read();
-                                            if (checkLength) length--;
-                                        }
-
-                                        outstream.flush();
-                                    } catch (IOException e) {
-                                        e.printStackTrace();  
-                                    }
-                                }
-                            });
-                            entity.setContentType("text/xml");
-                            response.setEntity(entity);
+                        InputStream instream = service.getClassLoader()
+                            .getResourceAsStream(DeploymentConstants.META_INF + "/" + schemaName);
+                        
+                        if (instream != null) {
+                            response.setStatus(HttpStatus.SC_OK);
+                            response.setContentType("text/xml");
+                            OutputStream outstream = response.getOutputStream();
+                            boolean checkLength = true;
+                            int length = Integer.MAX_VALUE;
+                            int nextValue = instream.read();
+                            if (checkLength) length--;
+                            while (-1 != nextValue && length >= 0) {
+                                outstream.write(nextValue);
+                                nextValue = instream.read();
+                                if (checkLength) length--;
+                            }
+                            outstream.flush();
                             return;
                         } else {
                             // no schema available by that name  - send 404
-                            response.setStatusLine(
-                                    new BasicStatusLine(ver, 404, "Schema Not Found!"));
+                            response.sendError(HttpStatus.SC_NOT_FOUND, "Schema Not Found!");
                             return;
                         }
                     }
                 }
             }
 
-            outbuffer = copyCommonProperties(msgContext, request);
-
             String contentType = null;
             Header[] headers = request.getHeaders(HTTPConstants.HEADER_CONTENT_TYPE);
             if (headers != null && headers.length > 0) {
@@ -228,74 +190,62 @@
                     contentType = contentType.substring(0, index);
                 }
             }
+            
             // deal with GET request
-            pi = RESTUtil.processURLRequest(msgContext, outbuffer.getOutputStream(), contentType);
+            pi = RESTUtil.processURLRequest(
+                    msgContext, 
+                    response.getOutputStream(), 
+                    contentType);
 
         } else if (method.equals(HTTPConstants.HEADER_POST)) {
             // deal with POST request
 
-            outbuffer = copyCommonProperties(msgContext, request);
-            HttpEntity inentity = ((HttpEntityEnclosingRequest) request).getEntity();
-            String contentType = processContentType(inentity, msgContext);
+            String contentType = request.getContentType();
+            
             if (HTTPTransportUtils.isRESTRequest(contentType)) {
-                pi = RESTUtil.processXMLRequest(msgContext, inentity.getContent(),
-                        outbuffer.getOutputStream(), contentType);
+                pi = RESTUtil.processXMLRequest(
+                        msgContext, 
+                        request.getInputStream(),
+                        response.getOutputStream(), 
+                        contentType);
             } else {
-                String ip = (String) msgContext.getProperty(MessageContext.TRANSPORT_ADDR);
-                if (ip != null) {
+                String ip = (String)msgContext.getProperty(MessageContext.TRANSPORT_ADDR);
+                if (ip != null){
                     uri = ip + uri;
                 }
-                pi = HTTPTransportUtils.processHTTPPostRequest(msgContext, inentity.getContent(),
-                        outbuffer.getOutputStream(),
-                        contentType, soapAction, uri);
+                pi = HTTPTransportUtils.processHTTPPostRequest(
+                        msgContext, 
+                        request.getInputStream(),
+                        response.getOutputStream(),
+                        contentType, 
+                        soapAction, 
+                        uri);
             }
 
-
         } else if (method.equals(HTTPConstants.HEADER_PUT)) {
-            outbuffer = copyCommonProperties(msgContext, request);
-            HttpEntity inentity = ((HttpEntityEnclosingRequest) request).getEntity();
-            String contentType = processContentType(inentity, msgContext);
-            pi = RESTUtil.processXMLRequest(msgContext, inentity.getContent(),
-                    outbuffer.getOutputStream(), contentType);
 
-        } else if (method.equals(HTTPConstants.HEADER_DELETE)) {
-            outbuffer = copyCommonProperties(msgContext, request);
+            String contentType = request.getContentType();
+            msgContext.setProperty(Constants.Configuration.CONTENT_TYPE, contentType);
+            
+            pi = RESTUtil.processXMLRequest(
+                    msgContext, 
+                    request.getInputStream(),
+                    response.getOutputStream(), 
+                    contentType);
 
+        } else if (method.equals(HTTPConstants.HEADER_DELETE)) {
 
-            pi = RESTUtil.processURLRequest(msgContext, outbuffer.getOutputStream(), null);
+            pi = RESTUtil.processURLRequest(
+                    msgContext, 
+                    response.getOutputStream(), 
+                    null);
 
         } else {
-
             throw new MethodNotSupportedException(method + " method not supported");
         }
-        handleResponse(pi, response, outbuffer, msgContext);
-        // Finalize response
-        OperationContext operationContext = msgContext.getOperationContext();
-        Object contextWritten = null;
-        Object isTwoChannel = null;
-        if (operationContext != null) {
-            contextWritten = operationContext.getProperty(Constants.RESPONSE_WRITTEN);
-            isTwoChannel = operationContext.getProperty(Constants.DIFFERENT_EPR);
-        }
-
-
-        if ((contextWritten != null) && Constants.VALUE_TRUE.equals(contextWritten)) {
-            if ((isTwoChannel != null) && Constants.VALUE_TRUE.equals(isTwoChannel)) {
-                response.setStatusLine(new BasicStatusLine(ver, 202, "OK"));
-                return;
-            }
-            response.setStatusLine(new BasicStatusLine(ver, 200, "OK"));
-        } else {
-            response.setStatusLine(new BasicStatusLine(ver, 202, "OK"));
-        }
-    }
-
-    private void handleResponse(InvocationResponse pi, HttpResponse response,
-                                OutputBuffer outbuffer, MessageContext msgContext)
-            throws IOException {
+        
         Boolean holdResponse =
-                (Boolean) msgContext.getProperty(RequestResponseTransport.HOLD_RESPONSE);
-
+            (Boolean) msgContext.getProperty(RequestResponseTransport.HOLD_RESPONSE);
         if (pi.equals(InvocationResponse.SUSPEND) ||
                 (holdResponse != null && Boolean.TRUE.equals(holdResponse))) {
             try {
@@ -307,32 +257,27 @@
                         e.getMessage());
             }
         }
+        
+        // Finalize response
+        OperationContext operationContext = msgContext.getOperationContext();
+        Object contextWritten = null;
+        Object isTwoChannel = null;
+        if (operationContext != null) {
+            contextWritten = operationContext.getProperty(Constants.RESPONSE_WRITTEN);
+            isTwoChannel = operationContext.getProperty(Constants.DIFFERENT_EPR);
+        }
 
-        response.setEntity(outbuffer);
-    }
 
-    private String processContentType(HttpEntity inentity, MessageContext msgContext) {
-        String contentType = null;
-        Header header = inentity.getContentType();
-        if (header != null) {
-            contentType = header.getValue();
+        if ((contextWritten != null) && Constants.VALUE_TRUE.equals(contextWritten)) {
+            if ((isTwoChannel != null) && Constants.VALUE_TRUE.equals(isTwoChannel)) {
+                response.setStatus(HttpStatus.SC_ACCEPTED);
+            }
+        } else {
+            response.setStatus(HttpStatus.SC_ACCEPTED);
         }
-        msgContext.setProperty(Constants.Configuration.CONTENT_TYPE, contentType);
-        return contentType;
     }
 
-    private OutputBuffer copyCommonProperties(MessageContext msgContext, HttpRequest request) {
-        request.getRequestLine().getUri();
-        OutputBuffer outbuffer = new OutputBuffer();
-        msgContext.setProperty(MessageContext.TRANSPORT_OUT, outbuffer);
-        msgContext.setProperty(Constants.OUT_TRANSPORT_INFO, outbuffer);
-        msgContext.setTo(new EndpointReference(request.getRequestLine().getUri()));
-        msgContext.setProperty(RequestResponseTransport.TRANSPORT_CONTROL,
-                new SimpleHTTPRequestResponseTransport());
-        return outbuffer;
-    }
-
-    public String getHostAddress(HttpRequest request) throws java.net.SocketException {
+    public String getHostAddress(AxisHttpRequest request) throws java.net.SocketException {
         try {
             Header hostHeader = request.getFirstHeader("host");
             if (hostHeader != null) {
@@ -345,38 +290,4 @@
         return HttpUtils.getIpAddress();
     }
 
-    class SimpleHTTPRequestResponseTransport implements RequestResponseTransport {
-
-        private CountDownLatch responseReadySignal = new CountDownLatch(1);
-        RequestResponseTransportStatus status = RequestResponseTransportStatus.INITIAL;
-        AxisFault faultToBeThrownOut = null;
-
-        public void acknowledgeMessage(MessageContext msgContext) throws AxisFault {
-            //TODO: Once the core HTTP API allows us to return an ack before unwinding, then the should be fixed
-            signalResponseReady();
-        }
-
-        public void awaitResponse() throws InterruptedException, AxisFault {
-            status = RequestResponseTransportStatus.WAITING;
-            responseReadySignal.await();
-
-            if (faultToBeThrownOut != null)
-                throw faultToBeThrownOut;
-        }
-
-        public void signalResponseReady() {
-            status = RequestResponseTransportStatus.SIGNALLED;
-            responseReadySignal.countDown();
-        }
-
-        public RequestResponseTransportStatus getStatus() {
-            return status;
-        }
-
-        public void signalFaultReady(AxisFault fault) {
-            faultToBeThrownOut = fault;
-            signalResponseReady();
-        }
-
-    }
 }

Added: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpConnection.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpConnection.java?view=auto&rev=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpConnection.java (added)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpConnection.java Thu Apr  5 19:12:24 2007
@@ -0,0 +1,61 @@
+/*
+ * $HeadURL$
+ * $Revision$
+ * $Date$
+ *
+ * ====================================================================
+ *
+ *  Copyright 1999-2006 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.
+ * ====================================================================
+ *
+ * This software consists of voluntary contributions made by many
+ * individuals on behalf of the Apache Software Foundation.  For more
+ * information on the Apache Software Foundation, please see
+ * <http://www.apache.org/>.
+ *
+ */
+
+package org.apache.axis2.transport.http.server;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+
+import org.apache.http.HttpConnection;
+import org.apache.http.HttpException;
+import org.apache.http.HttpInetConnection;
+import org.apache.http.HttpRequest;
+import org.apache.http.HttpResponse;
+import org.apache.http.params.HttpParams;
+
+public interface AxisHttpConnection extends HttpConnection, HttpInetConnection {
+
+    HttpRequest receiveRequest(HttpParams params)
+        throws HttpException, IOException;
+    
+    InputStream getInputStream();
+    
+    void sendResponse(HttpResponse response) 
+        throws HttpException, IOException;    
+    
+    OutputStream getOutputStream();
+    
+    void flush()
+        throws IOException;
+    
+    void reset()
+        throws IOException;
+
+}

Added: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpConnectionImpl.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpConnectionImpl.java?view=auto&rev=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpConnectionImpl.java (added)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpConnectionImpl.java Thu Apr  5 19:12:24 2007
@@ -0,0 +1,323 @@
+/*
+ * $HeadURL$
+ * $Revision$
+ * $Date$
+ *
+ * ====================================================================
+ *
+ *  Copyright 1999-2006 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.
+ * ====================================================================
+ *
+ * This software consists of voluntary contributions made by many
+ * individuals on behalf of the Apache Software Foundation.  For more
+ * information on the Apache Software Foundation, please see
+ * <http://www.apache.org/>.
+ *
+ */
+
+package org.apache.axis2.transport.http.server;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.Socket;
+import java.net.SocketException;
+import java.util.Iterator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.http.ConnectionClosedException;
+import org.apache.http.Header;
+import org.apache.http.HttpEntity;
+import org.apache.http.HttpEntityEnclosingRequest;
+import org.apache.http.HttpException;
+import org.apache.http.HttpRequest;
+import org.apache.http.HttpRequestFactory;
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpVersion;
+import org.apache.http.RequestLine;
+import org.apache.http.entity.ContentLengthStrategy;
+import org.apache.http.impl.DefaultHttpRequestFactory;
+import org.apache.http.impl.entity.StrictContentLengthStrategy;
+import org.apache.http.impl.io.ChunkedInputStream;
+import org.apache.http.impl.io.ChunkedOutputStream;
+import org.apache.http.impl.io.ContentLengthInputStream;
+import org.apache.http.impl.io.ContentLengthOutputStream;
+import org.apache.http.impl.io.HttpDataInputStream;
+import org.apache.http.impl.io.IdentityOutputStream;
+import org.apache.http.impl.io.SocketHttpDataReceiver;
+import org.apache.http.impl.io.SocketHttpDataTransmitter;
+import org.apache.http.io.HttpDataReceiver;
+import org.apache.http.io.HttpDataTransmitter;
+import org.apache.http.message.BasicHeader;
+import org.apache.http.message.BasicRequestLine;
+import org.apache.http.message.BasicStatusLine;
+import org.apache.http.message.BufferedHeader;
+import org.apache.http.params.HttpConnectionParams;
+import org.apache.http.params.HttpParams;
+import org.apache.http.util.CharArrayBuffer;
+import org.apache.http.util.HeaderUtils;
+
+public class AxisHttpConnectionImpl implements AxisHttpConnection {
+
+    private static final Log HEADERLOG =
+        LogFactory.getLog("org.apache.axis2.transport.http.server.wire");
+
+    private final Socket socket;
+    private final HttpDataTransmitter datatransmitter;
+    private final HttpDataReceiver datareceiver;
+    private final CharArrayBuffer charbuffer; 
+    private final HttpRequestFactory requestfactory;
+    private final ContentLengthStrategy contentLenStrategy;
+    private final int maxHeaderCount;
+
+    private OutputStream out = null;
+    private InputStream in = null;
+    
+    public AxisHttpConnectionImpl(final Socket socket, final HttpParams params) 
+            throws IOException {
+        super();
+        if (socket == null) {
+            throw new IllegalArgumentException("Socket may not be null"); 
+        }
+        if (params == null) {
+            throw new IllegalArgumentException("HTTP parameters may not be null"); 
+        }
+        socket.setTcpNoDelay(HttpConnectionParams.getTcpNoDelay(params));
+        socket.setSoTimeout(HttpConnectionParams.getSoTimeout(params));
+        
+        int linger = HttpConnectionParams.getLinger(params);
+        if (linger >= 0) {
+            socket.setSoLinger(linger > 0, linger);
+        }
+        
+        int buffersize = HttpConnectionParams.getSocketBufferSize(params);
+        this.socket = socket;
+        this.datatransmitter = new SocketHttpDataTransmitter(socket, buffersize); 
+        this.datareceiver = new SocketHttpDataReceiver(socket, buffersize); 
+        this.charbuffer = new CharArrayBuffer(256);
+        this.requestfactory = new DefaultHttpRequestFactory();
+        this.contentLenStrategy = new StrictContentLengthStrategy();
+        this.maxHeaderCount = params.getIntParameter(HttpConnectionParams.MAX_HEADER_COUNT, -1);
+    }
+
+    public void close() throws IOException {
+        this.datatransmitter.flush();
+        try {
+            this.socket.shutdownOutput();
+        } catch (IOException ignore) {
+        }
+        try {
+            this.socket.shutdownInput();
+        } catch (IOException ignore) {
+        }
+        this.socket.close();
+    }
+
+    public boolean isOpen() {
+        return !this.socket.isClosed();
+    }
+
+    public boolean isStale() {
+        try {
+            this.datareceiver.isDataAvailable(1);
+            return false;
+        } catch (IOException ex) {
+            return true;
+        }
+    }
+
+    public void shutdown() throws IOException {
+        Socket tmpsocket = this.socket;
+        if (tmpsocket != null) {
+            tmpsocket.close();
+        }
+    }
+
+    public HttpRequest receiveRequest(final HttpParams params) 
+            throws HttpException, IOException {
+        if (params == null) {
+            throw new IllegalArgumentException("HTTP parameters may not be null");
+        }
+        this.charbuffer.clear();
+        int i = this.datareceiver.readLine(this.charbuffer);
+        if (i == -1) {
+            throw new ConnectionClosedException("Client closed connection"); 
+        }
+        RequestLine requestline = BasicRequestLine.parse(this.charbuffer, 0, this.charbuffer.length());
+        HttpRequest request = this.requestfactory.newHttpRequest(requestline);
+        request.getParams().setDefaults(params);
+        Header[] headers = HeaderUtils.parseHeaders(this.datareceiver, this.maxHeaderCount);
+        request.setHeaders(headers);
+        
+        if (HEADERLOG.isDebugEnabled()) {
+            HEADERLOG.debug(">> " + request.getRequestLine().toString());
+            for (i = 0; i < headers.length; i++) {
+                HEADERLOG.debug(">> " + headers[i].toString());
+            }
+        }
+        
+        // Prepare input stream
+        this.in = null;
+        if (request instanceof HttpEntityEnclosingRequest) {
+            long len = this.contentLenStrategy.determineLength(request);
+            if (len == ContentLengthStrategy.CHUNKED) {
+                this.in = new ChunkedInputStream(this.datareceiver);
+            } else if (len == ContentLengthStrategy.IDENTITY) {
+                this.in = new HttpDataInputStream(this.datareceiver);                            
+            } else {
+                this.in = new ContentLengthInputStream(datareceiver, len);
+            }
+        }
+        return request;
+    }
+    
+    public void sendResponse(final HttpResponse response) 
+            throws HttpException, IOException {
+        if (response == null) {
+            throw new IllegalArgumentException("HTTP response may not be null");
+        }
+
+        if (HEADERLOG.isDebugEnabled()) {
+            HEADERLOG.debug("<< " + response.getStatusLine().toString());
+            Header[] headers = response.getAllHeaders();
+            for (int i = 0; i < headers.length; i++) {
+                HEADERLOG.debug("<< " + headers[i].toString());
+            }
+        }
+        
+        this.charbuffer.clear();
+        BasicStatusLine.format(this.charbuffer, response.getStatusLine());
+        this.datatransmitter.writeLine(this.charbuffer);
+        for (Iterator it = response.headerIterator(); it.hasNext(); ) {
+            Header header = (Header) it.next();
+            if (header instanceof BufferedHeader) {
+                this.datatransmitter.writeLine(((BufferedHeader)header).getBuffer());
+            } else {
+                this.charbuffer.clear();
+                BasicHeader.format(this.charbuffer, header);
+                this.datatransmitter.writeLine(this.charbuffer);
+            }
+        }
+        this.charbuffer.clear();
+        this.datatransmitter.writeLine(this.charbuffer);
+
+        // Prepare output stream
+        this.out = null;
+        HttpVersion ver = response.getStatusLine().getHttpVersion();
+        HttpEntity entity = response.getEntity();
+        if (entity != null) {
+            long len = entity.getContentLength();
+            if (entity.isChunked() && ver.greaterEquals(HttpVersion.HTTP_1_1)) {
+                this.out = new ChunkedOutputStream(this.datatransmitter);
+            } else if (len >= 0) {
+                this.out = new ContentLengthOutputStream(this.datatransmitter, len);
+            } else {
+                this.out = new IdentityOutputStream(this.datatransmitter); 
+            }
+        } else {
+            this.datatransmitter.flush();
+        }
+    }
+    
+    public InputStream getInputStream() {
+        return this.in;
+    }
+
+    public OutputStream getOutputStream() {
+        return this.out;
+    }
+    
+    public void flush() throws IOException {
+        if (this.out != null) {
+            this.out.flush();
+        } else {
+            this.datatransmitter.flush();
+        }
+    }
+
+    public void reset() throws IOException {
+        if (this.in != null) {
+            this.in.close();
+            this.in = null;
+        }
+        if (this.out != null) {
+            this.out.flush();
+            this.out.close();
+            this.out = null;
+        }
+    }
+    
+    public int getSocketTimeout() {
+        try {
+            return this.socket.getSoTimeout();
+        } catch (SocketException ex) {
+            return -1;
+        }
+    }
+
+    public void setSocketTimeout(int timeout) {
+        try {
+            this.socket.setSoTimeout(timeout);
+        } catch (SocketException ex) {
+        }
+    }
+
+    public InetAddress getLocalAddress() {
+        if (this.socket != null) {
+            return this.socket.getLocalAddress();
+        } else {
+            return null;
+        }
+    }
+
+    public int getLocalPort() {
+        if (this.socket != null) {
+            return this.socket.getLocalPort();
+        } else {
+            return -1;
+        }
+    }
+
+    public InetAddress getRemoteAddress() {
+        if (this.socket != null) {
+            return this.socket.getInetAddress();
+        } else {
+            return null;
+        }
+    }
+
+    public int getRemotePort() {
+        if (this.socket != null) {
+            return this.socket.getPort();
+        } else {
+            return -1;
+        }
+    }
+
+    public String toString() {
+        StringBuffer buffer = new StringBuffer();
+        buffer.append("[");
+        if (isOpen()) {
+            buffer.append(this.socket.getInetAddress());
+        } else {
+            buffer.append("closed");
+        }
+        buffer.append("]");
+        return buffer.toString();
+    }
+
+}

Added: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpRequest.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpRequest.java?view=auto&rev=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpRequest.java (added)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpRequest.java Thu Apr  5 19:12:24 2007
@@ -0,0 +1,46 @@
+/*
+ * $HeadURL$
+ * $Revision$
+ * $Date$
+ *
+ * ====================================================================
+ *
+ *  Copyright 1999-2006 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.
+ * ====================================================================
+ *
+ * This software consists of voluntary contributions made by many
+ * individuals on behalf of the Apache Software Foundation.  For more
+ * information on the Apache Software Foundation, please see
+ * <http://www.apache.org/>.
+ *
+ */
+
+package org.apache.axis2.transport.http.server;
+
+import java.io.InputStream;
+
+import org.apache.http.HttpMessage;
+
+public interface AxisHttpRequest extends HttpMessage {
+
+    String getMethod();
+    
+    String getRequestURI();
+    
+    String getContentType();
+    
+    InputStream getInputStream();
+    
+}

Added: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpRequestImpl.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpRequestImpl.java?view=auto&rev=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpRequestImpl.java (added)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpRequestImpl.java Thu Apr  5 19:12:24 2007
@@ -0,0 +1,169 @@
+/*
+ * $HeadURL$
+ * $Revision$
+ * $Date$
+ *
+ * ====================================================================
+ *
+ *  Copyright 1999-2006 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.
+ * ====================================================================
+ *
+ * This software consists of voluntary contributions made by many
+ * individuals on behalf of the Apache Software Foundation.  For more
+ * information on the Apache Software Foundation, please see
+ * <http://www.apache.org/>.
+ *
+ */
+
+package org.apache.axis2.transport.http.server;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Iterator;
+
+import org.apache.http.Header;
+import org.apache.http.HttpException;
+import org.apache.http.HttpRequest;
+import org.apache.http.HttpVersion;
+import org.apache.http.params.HttpParams;
+import org.apache.http.protocol.HTTP;
+import org.apache.http.protocol.HttpContext;
+import org.apache.http.protocol.HttpExecutionContext;
+import org.apache.http.protocol.HttpProcessor;
+
+public class AxisHttpRequestImpl implements AxisHttpRequest {
+
+    private final HttpRequest request;
+    private final AxisHttpConnection conn;
+    private final HttpProcessor httpproc;
+    private final HttpContext context;
+    
+    public AxisHttpRequestImpl(
+            final AxisHttpConnection conn,
+            final HttpRequest request, 
+            final HttpProcessor httpproc,
+            final HttpContext context) {
+        super();
+        if (conn == null) {
+            throw new IllegalArgumentException("HTTP connection may not be null");
+        }
+        if (request == null) {
+            throw new IllegalArgumentException("HTTP request may not be null");
+        }
+        if (httpproc == null) {
+            throw new IllegalArgumentException("HTTP processor may not be null");
+        }
+        if (context == null) {
+            throw new IllegalArgumentException("HTTP context may not be null");
+        }
+        this.request = request;
+        this.conn = conn;
+        this.httpproc = httpproc;
+        this.context = context;
+    }
+    
+    public void prepare() throws IOException, HttpException {
+        this.context.setAttribute(HttpExecutionContext.HTTP_CONNECTION, this.conn);
+        this.context.setAttribute(HttpExecutionContext.HTTP_REQUEST, this.request);
+        
+        this.httpproc.process(this.request, this.context);
+    }
+
+    public String getMethod() {
+        return this.request.getRequestLine().getMethod();
+    }
+
+    public String getRequestURI() {
+        return this.request.getRequestLine().getUri();
+    }
+
+    public HttpVersion getHttpVersion() {
+        return this.request.getRequestLine().getHttpVersion();
+    }
+
+    public String getContentType() {
+        Header header = this.request.getFirstHeader(HTTP.CONTENT_TYPE);
+        if (header != null) {
+            return header.getValue();
+        } else {
+            return null;
+        }
+    }
+
+    public InputStream getInputStream() {
+        return this.conn.getInputStream();
+    }
+
+    public void addHeader(final Header header) {
+        this.request.addHeader(header);
+    }
+
+    public void addHeader(final String name, final String value) {
+        this.request.addHeader(name, value);
+    }
+
+    public boolean containsHeader(final String name) {
+        return this.request.containsHeader(name);
+    }
+
+    public Header[] getAllHeaders() {
+        return this.request.getAllHeaders();
+    }
+
+    public Header getFirstHeader(final String name) {
+        return this.request.getFirstHeader(name);
+    }
+
+    public Header[] getHeaders(String name) {
+        return this.request.getHeaders(name);
+    }
+
+    public Header getLastHeader(final String name) {
+        return this.request.getLastHeader(name);
+    }
+
+    public Iterator headerIterator() {
+        return this.request.headerIterator();
+    }
+
+    public void removeHeader(final Header header) {
+        this.request.removeHeader(header);
+    }
+
+    public void removeHeaders(final String name) {
+        this.request.removeHeaders(name);
+    }
+
+    public void setHeader(final Header header) {
+        this.request.setHeader(header);
+    }
+
+    public void setHeader(final String name, final String value) {
+        this.request.setHeader(name, value);
+    }
+
+    public void setHeaders(Header[] headers) {
+        this.request.setHeaders(headers);
+    }
+
+    public HttpParams getParams() {
+        return this.request.getParams();
+    }
+
+    public void setParams(final HttpParams params) {
+        this.request.setParams(params);
+    }
+    
+}

Added: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpResponse.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpResponse.java?view=auto&rev=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpResponse.java (added)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpResponse.java Thu Apr  5 19:12:24 2007
@@ -0,0 +1,48 @@
+/*
+ * $HeadURL$
+ * $Revision$
+ * $Date$
+ *
+ * ====================================================================
+ *
+ *  Copyright 1999-2006 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.
+ * ====================================================================
+ *
+ * This software consists of voluntary contributions made by many
+ * individuals on behalf of the Apache Software Foundation.  For more
+ * information on the Apache Software Foundation, please see
+ * <http://www.apache.org/>.
+ *
+ */
+
+package org.apache.axis2.transport.http.server;
+
+import java.io.OutputStream;
+
+import org.apache.http.HttpMessage;
+
+public interface AxisHttpResponse extends HttpMessage {
+
+    void setStatus(int sc);
+
+    void sendError(int sc, String msg);
+    
+    void sendError(int sc);
+
+    void setContentType(String contentType);
+    
+    OutputStream getOutputStream();
+    
+}

Added: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpResponseImpl.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpResponseImpl.java?view=auto&rev=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpResponseImpl.java (added)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpResponseImpl.java Thu Apr  5 19:12:24 2007
@@ -0,0 +1,257 @@
+/*
+ * $HeadURL$
+ * $Revision$
+ * $Date$
+ *
+ * ====================================================================
+ *
+ *  Copyright 1999-2006 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.
+ * ====================================================================
+ *
+ * This software consists of voluntary contributions made by many
+ * individuals on behalf of the Apache Software Foundation.  For more
+ * information on the Apache Software Foundation, please see
+ * <http://www.apache.org/>.
+ *
+ */
+
+package org.apache.axis2.transport.http.server;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.Iterator;
+
+import org.apache.axis2.transport.OutTransportInfo;
+import org.apache.http.Header;
+import org.apache.http.HttpException;
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpVersion;
+import org.apache.http.entity.BasicHttpEntity;
+import org.apache.http.params.HttpParams;
+import org.apache.http.protocol.HttpContext;
+import org.apache.http.protocol.HttpExecutionContext;
+import org.apache.http.protocol.HttpProcessor;
+
+public class AxisHttpResponseImpl implements AxisHttpResponse, OutTransportInfo {
+
+    private final HttpResponse response;
+    private final AxisHttpConnection conn;
+    private final HttpProcessor httpproc;
+    private final HttpContext context;
+    
+    private AutoCommitOutputStream outstream;
+    private String contentType;
+    
+    private volatile boolean commited;
+    
+    public AxisHttpResponseImpl(
+            final AxisHttpConnection conn,
+            final HttpResponse response, 
+            final HttpProcessor httpproc,
+            final HttpContext context) {
+        super();
+        if (response == null) {
+            throw new IllegalArgumentException("HTTP response may not be null");
+        }
+        if (conn == null) {
+            throw new IllegalArgumentException("HTTP connection may not be null");
+        }
+        if (httpproc == null) {
+            throw new IllegalArgumentException("HTTP processor may not be null");
+        }
+        if (context == null) {
+            throw new IllegalArgumentException("HTTP context may not be null");
+        }
+        this.response = response;
+        this.conn = conn;
+        this.httpproc = httpproc;
+        this.context = context;
+    }
+
+    private void assertNotCommitted() {
+        if (this.commited) {
+            throw new IllegalStateException("Response already committed");
+        }
+    }
+    
+    public boolean isCommitted() {
+        return this.commited;
+    }
+    
+    public void commit() throws IOException, HttpException {
+        if (this.commited) {
+            return;
+        }
+        this.commited = true;
+        
+        this.context.setAttribute(HttpExecutionContext.HTTP_CONNECTION, this.conn);
+        this.context.setAttribute(HttpExecutionContext.HTTP_RESPONSE, this.response);
+        
+        BasicHttpEntity entity = new BasicHttpEntity();
+        entity.setChunked(true);
+        entity.setContentType(this.contentType);
+        
+        this.response.setEntity(entity);
+        
+        this.httpproc.process(this.response, this.context);
+        this.conn.sendResponse(this.response);
+    }
+    
+    public OutputStream getOutputStream() {
+        if (this.outstream == null) {
+            this.outstream = new AutoCommitOutputStream();
+        }
+        return this.outstream;
+    }
+
+    public void sendError(int sc, final String msg) {
+        assertNotCommitted();
+        HttpVersion ver = this.response.getHttpVersion();
+        this.response.setStatusLine(ver, sc, msg);
+    }
+
+    public void sendError(int sc) {
+        assertNotCommitted();
+        this.response.setStatusCode(sc);
+    }
+
+    public void setStatus(int sc) {
+        assertNotCommitted();
+        this.response.setStatusCode(sc);
+    }
+
+    public void setContentType(final String contentType) {
+        assertNotCommitted();
+        this.contentType = contentType;
+    }
+
+    public HttpVersion getHttpVersion() {
+        return this.response.getHttpVersion();
+    }
+
+    public void addHeader(final Header header) {
+        assertNotCommitted();
+        this.response.addHeader(header);
+    }
+
+    public void addHeader(final String name, final String value) {
+        assertNotCommitted();
+        this.response.addHeader(name, value);
+    }
+
+    public boolean containsHeader(final String name) {
+        return this.response.containsHeader(name);
+    }
+
+    public Header[] getAllHeaders() {
+        return this.response.getAllHeaders();
+    }
+
+    public Header getFirstHeader(final String name) {
+        return this.response.getFirstHeader(name);
+    }
+
+    public Header[] getHeaders(String name) {
+        return this.response.getHeaders(name);
+    }
+
+    public Header getLastHeader(final String name) {
+        return this.response.getLastHeader(name);
+    }
+
+    public Iterator headerIterator() {
+        return this.response.headerIterator();
+    }
+
+    public void removeHeader(final Header header) {
+        assertNotCommitted();
+        this.response.removeHeader(header);
+    }
+
+    public void removeHeaders(final String name) {
+        assertNotCommitted();
+        this.response.removeHeaders(name);
+    }
+
+    public void setHeader(final Header header) {
+        assertNotCommitted();
+        this.response.setHeader(header);
+    }
+
+    public void setHeader(final String name, final String value) {
+        assertNotCommitted();
+        this.response.setHeader(name, value);
+    }
+
+    public void setHeaders(Header[] headers) {
+        assertNotCommitted();
+        this.response.setHeaders(headers);
+    }
+
+    public HttpParams getParams() {
+        return this.response.getParams();
+    }
+
+    public void setParams(final HttpParams params) {
+        this.response.setParams(params);
+    }
+    
+    class AutoCommitOutputStream extends OutputStream {
+
+        private OutputStream out;
+        
+        public AutoCommitOutputStream() {
+            super();
+        }
+
+        private void ensureCommitted() throws IOException {
+            try {
+                commit();
+            } catch (HttpException ex) {
+                throw new IOException("HTTP protocol exception: " + ex.getMessage());
+            }
+            if (this.out == null) {
+                this.out = conn.getOutputStream();
+            }
+        }
+        
+        public void close() throws IOException {
+            ensureCommitted();
+            this.out.close();
+        }
+
+        public void write(final byte[] b, int off, int len) throws IOException {
+            ensureCommitted();
+            this.out.write(b, off, len);
+        }
+
+        public void write(final byte[] b) throws IOException {
+            ensureCommitted();
+            this.out.write(b);
+        }
+
+        public void write(int b) throws IOException {
+            ensureCommitted();
+            this.out.write(b);
+        }
+        
+        public void flush() throws IOException {
+            ensureCommitted();
+            this.out.flush();
+        }
+
+    }
+    
+}

Modified: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpService.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpService.java?view=diff&rev=526026&r1=526025&r2=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpService.java (original)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/AxisHttpService.java Thu Apr  5 19:12:24 2007
@@ -27,53 +27,64 @@
 */
 package org.apache.axis2.transport.http.server;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.SocketException;
+import java.util.HashMap;
+import java.util.Iterator;
+
+import org.apache.axis2.AxisFault;
 import org.apache.axis2.Constants;
 import org.apache.axis2.addressing.AddressingHelper;
+import org.apache.axis2.addressing.EndpointReference;
 import org.apache.axis2.context.ConfigurationContext;
 import org.apache.axis2.context.ContextFactory;
 import org.apache.axis2.context.MessageContext;
 import org.apache.axis2.description.TransportInDescription;
 import org.apache.axis2.description.TransportOutDescription;
 import org.apache.axis2.engine.AxisEngine;
+import org.apache.axis2.transport.RequestResponseTransport;
 import org.apache.axis2.transport.http.HTTPConstants;
 import org.apache.axis2.util.MessageContextBuilder;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.http.ConnectionReuseStrategy;
 import org.apache.http.Header;
+import org.apache.http.HttpEntityEnclosingRequest;
 import org.apache.http.HttpException;
 import org.apache.http.HttpInetConnection;
 import org.apache.http.HttpRequest;
 import org.apache.http.HttpResponse;
 import org.apache.http.HttpResponseFactory;
-import org.apache.http.HttpServerConnection;
+import org.apache.http.HttpStatus;
 import org.apache.http.HttpVersion;
-import org.apache.http.RequestLine;
+import org.apache.http.MethodNotSupportedException;
+import org.apache.http.ProtocolException;
 import org.apache.http.UnsupportedHttpVersionException;
-import org.apache.http.entity.StringEntity;
-import org.apache.http.message.BasicStatusLine;
+import org.apache.http.params.HttpParams;
 import org.apache.http.protocol.HttpContext;
 import org.apache.http.protocol.HttpProcessor;
-import org.apache.http.protocol.HttpService;
 
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.SocketException;
-import java.util.HashMap;
-import java.util.Iterator;
+import edu.emory.mathcs.backport.java.util.concurrent.CountDownLatch;
 
 /**
  * This class is an extension of the defaulf HTTP service responsible for
  * maintaining and polulating the {@link MessageContext} for incoming Axis
  * requests.
  */
-public class AxisHttpService extends HttpService {
+public class AxisHttpService {
 
     private static final Log LOG = LogFactory.getLog(AxisHttpService.class);
 
+    private final HttpProcessor httpProcessor;
+    private final ConnectionReuseStrategy connStrategy;
+    private final HttpResponseFactory responseFactory;
+    private final MessageContext msgContext;
     private final ConfigurationContext configurationContext;
     private final Worker worker;
-    private HttpServerConnection conn;
+
+    private HttpParams params;
 
     public AxisHttpService(
             final HttpProcessor httpProcessor,
@@ -81,45 +92,151 @@
             final HttpResponseFactory responseFactory,
             final ConfigurationContext configurationContext,
             final Worker worker) {
-        super(httpProcessor, connStrategy, responseFactory);
+        super();
+        if (httpProcessor == null) {
+            throw new IllegalArgumentException("HTTP processor may not be null");
+        }
+        if (connStrategy == null) {
+            throw new IllegalArgumentException("Connection strategy may not be null");
+        }
+        if (responseFactory == null) {
+            throw new IllegalArgumentException("Response factory may not be null");
+        }
         if (worker == null) {
             throw new IllegalArgumentException("Worker may not be null");
         }
         if (configurationContext == null) {
             throw new IllegalArgumentException("Configuration context may not be null");
         }
+        this.httpProcessor = httpProcessor;
+        this.connStrategy = connStrategy;
+        this.responseFactory = responseFactory;
         this.configurationContext = configurationContext;
         this.worker = worker;
+
+        this.msgContext = ContextFactory.createMessageContext(configurationContext);
+        this.msgContext.setIncomingTransportName(Constants.TRANSPORT_HTTP);
     }
 
-    protected void doService(
-            final HttpRequest request,
-            final HttpResponse response,
-            final HttpContext context) throws HttpException, IOException {
-        RequestLine reqline = request.getRequestLine();
-        MessageContext msgContext = ContextFactory.createMessageContext(configurationContext);
-        msgContext.setIncomingTransportName(Constants.TRANSPORT_HTTP);
-        if(conn!=null){
-            if (conn instanceof HttpInetConnection) {
-                HttpInetConnection inetconn = (HttpInetConnection) conn;
-                InetAddress address = inetconn.getRemoteAddress();
-                msgContext.setProperty(MessageContext.REMOTE_ADDR, address.getHostAddress());
+    public HttpParams getParams() {
+        return this.params;
+    }
+    
+    public void setParams(final HttpParams params) {
+        this.params = params;
+    }
+    
+    public void handleRequest(final AxisHttpConnection conn, final HttpContext context) 
+            throws IOException, HttpException { 
+        
+        if (conn instanceof HttpInetConnection) {
+            HttpInetConnection inetconn = (HttpInetConnection) conn;
+            this.msgContext.setProperty(MessageContext.REMOTE_ADDR, 
+                    inetconn.getRemoteAddress().getHostAddress());
+            this.msgContext.setProperty(MessageContext.TRANSPORT_ADDR,
+                    inetconn.getLocalAddress().getHostAddress());
+
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Remote address of the connection : " + 
+                        inetconn.getRemoteAddress().getHostAddress());
+            }
+        }
+        
+        HttpResponse response;
+        try {
+            HttpRequest request = conn.receiveRequest(this.params);
+            HttpVersion ver = request.getRequestLine().getHttpVersion();
+            if (!ver.lessEquals(HttpVersion.HTTP_1_1)) {
+                // Downgrade protocol version if greater than HTTP/1.1 
+                ver = HttpVersion.HTTP_1_1;
+            }
 
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("Remote address of the connection : " + address);
+            response = this.responseFactory.newHttpResponse
+                (ver, HttpStatus.SC_OK, context);
+            response.getParams().setDefaults(this.params);
+            
+            if (request instanceof HttpEntityEnclosingRequest) {
+                if (((HttpEntityEnclosingRequest) request).expectContinue()) {
+                    HttpResponse ack = this.responseFactory.newHttpResponse
+                        (ver, HttpStatus.SC_CONTINUE, context);
+                    ack.getParams().setDefaults(this.params);
+                    conn.sendResponse(ack);
+                    conn.flush();
                 }
             }
-            msgContext.setProperty(MessageContext.TRANSPORT_ADDR,
-                    ((DefaultHttpConnectionFactory.Axis2HttpServerConnection)conn).getLocalIPAddress());
+            
+            // Create Axis request and response objects
+            AxisHttpRequestImpl axisreq = new AxisHttpRequestImpl(
+                    conn, 
+                    request, 
+                    this.httpProcessor, 
+                    context); 
+            AxisHttpResponseImpl axisres = new AxisHttpResponseImpl(
+                    conn, 
+                    response, 
+                    this.httpProcessor, 
+                    context); 
+
+            // Prepare HTTP request
+            axisreq.prepare();
+            
+            // Run the service
+            doService(axisreq, axisres, context);
+            
+            // Make sure the request content is fully consumed
+            InputStream instream = conn.getInputStream();
+            if (instream != null) {
+                instream.close();
+            }
+
+            // Commit response if not committed
+            if (!axisres.isCommitted()) {
+                axisres.commit();
+            }
+
+            // Make sure the response content is properly terminated
+            OutputStream outstream = conn.getOutputStream();
+            if (outstream != null) {
+                outstream.close();
+            }
+            
+        } catch (HttpException ex) {
+            response = this.responseFactory.newHttpResponse
+                (HttpVersion.HTTP_1_0, HttpStatus.SC_INTERNAL_SERVER_ERROR,
+                 context);
+            response.getParams().setDefaults(this.params);
+            handleException(ex, response);
+            this.httpProcessor.process(response, context);
+            conn.sendResponse(response);
         }
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Request method: " + reqline.getMethod());
-            LOG.debug("Target URI: " + reqline.getUri());
+        
+        conn.flush();
+        if (!this.connStrategy.keepAlive(response, context)) {
+            conn.close();
+        } else {
+            conn.reset();
         }
-
-        HttpVersion ver = reqline.getHttpVersion();
-        if (!ver.lessEquals(HttpVersion.HTTP_1_1)) {
-            throw new UnsupportedHttpVersionException("Unsupported HTTP version: " + ver);
+    }    
+    
+    protected void handleException(final HttpException ex, final HttpResponse response) {
+        if (ex instanceof MethodNotSupportedException) {
+            response.setStatusCode(HttpStatus.SC_NOT_IMPLEMENTED);
+        } else if (ex instanceof UnsupportedHttpVersionException) {
+            response.setStatusCode(HttpStatus.SC_HTTP_VERSION_NOT_SUPPORTED);
+        } else if (ex instanceof ProtocolException) {
+            response.setStatusCode(HttpStatus.SC_BAD_REQUEST);
+        } else {
+            response.setStatusCode(HttpStatus.SC_INTERNAL_SERVER_ERROR);
+        }
+    }
+       
+    protected void doService(
+            final AxisHttpRequest request, 
+            final AxisHttpResponse response,
+            final HttpContext context) throws HttpException, IOException {
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Request method: " + request.getMethod());
+            LOG.debug("Target URI: " + request.getRequestURI());
         }
 
         try {
@@ -128,12 +245,13 @@
             TransportInDescription transportIn = this.configurationContext.getAxisConfiguration()
                     .getTransportIn(Constants.TRANSPORT_HTTP);
 
-   ;         String sessionKey = (String) context.getAttribute(HTTPConstants.COOKIE_STRING);
-            msgContext.setTransportIn(transportIn);
-            msgContext.setTransportOut(transportOut);
-            msgContext.setServerSide(true);
-            msgContext.setProperty(HTTPConstants.COOKIE_STRING, sessionKey);
-            msgContext.setProperty(Constants.Configuration.TRANSPORT_IN_URL, reqline.getUri());
+            String sessionKey = (String) context.getAttribute(HTTPConstants.COOKIE_STRING);
+            this.msgContext.setTransportIn(transportIn);
+            this.msgContext.setTransportOut(transportOut);
+            this.msgContext.setServerSide(true);
+            this.msgContext.setProperty(HTTPConstants.COOKIE_STRING, sessionKey);
+            this.msgContext.setProperty(Constants.Configuration.TRANSPORT_IN_URL, 
+                    request.getRequestURI());
 
             // set the transport Headers
             HashMap headerMap = new HashMap();
@@ -141,8 +259,20 @@
                 Header header = (Header) it.next();
                 headerMap.put(header.getName(), header.getValue());
             }
-            msgContext.setProperty(MessageContext.TRANSPORT_HEADERS, headerMap);
-            this.worker.service(request, response, msgContext);
+            this.msgContext.setProperty(MessageContext.TRANSPORT_HEADERS, 
+                    headerMap);
+            this.msgContext.setProperty(Constants.Configuration.CONTENT_TYPE, 
+                    request.getContentType());
+            
+            this.msgContext.setProperty(MessageContext.TRANSPORT_OUT, 
+                    response.getOutputStream());
+            this.msgContext.setProperty(Constants.OUT_TRANSPORT_INFO, 
+                    response);
+            this.msgContext.setTo(new EndpointReference(request.getRequestURI()));
+            this.msgContext.setProperty(RequestResponseTransport.TRANSPORT_CONTROL,
+                                 new SimpleHTTPRequestResponseTransport());
+            
+            this.worker.service(request, response, this.msgContext);
         } catch (SocketException ex) {
             // Socket is unreliable. 
             throw ex;
@@ -150,49 +280,60 @@
             // HTTP protocol violation. Transport is unrelaible
             throw ex;
         } catch (Throwable e) {
-            try {
-                AxisEngine engine = new AxisEngine(this.configurationContext);
 
-                OutputBuffer outbuffer = new OutputBuffer();
-                msgContext
-                        .setProperty(MessageContext.TRANSPORT_OUT, outbuffer.getOutputStream());
-                msgContext.setProperty(Constants.OUT_TRANSPORT_INFO, outbuffer);
-
-                MessageContext faultContext =
-                        MessageContextBuilder.createFaultMessageContext(msgContext, e);
-                // If the fault is not going along the back channel we should be 202ing
-                if (AddressingHelper.isFaultRedirected(msgContext)) {
-                    response.setStatusLine(new BasicStatusLine(ver, 202, "Accepted"));
-                } else {
-                    response.setStatusLine(new BasicStatusLine(ver, 500, "Internal server error"));
-                }
-                engine.sendFault(faultContext);
-                response.setEntity(outbuffer);
-            } catch (Exception ex) {
-                if (AddressingHelper.isFaultRedirected(msgContext)) {
-                    response.setStatusLine(new BasicStatusLine(ver, 202, "Accepted"));
-                } else {
-                    // TODO: Why isn't this a SOAP fault?
-                    response.setStatusLine(new BasicStatusLine(ver, 500, "Internal server error"));
-                    String msg = ex.getMessage();
-                    StringEntity entity;
-                    if (msg != null && msg.trim().length() != 0) {
-                        entity = new StringEntity(msg);
-                    } else {
-                        entity = new StringEntity("Exception message unknown");
-                    }
-                    entity.setContentType("text/plain");
-                    response.setEntity(entity);
-                }
+            AxisEngine engine = new AxisEngine(this.configurationContext);
+
+            this.msgContext.setProperty(MessageContext.TRANSPORT_OUT, 
+                    response.getOutputStream());
+            this.msgContext.setProperty(Constants.OUT_TRANSPORT_INFO, 
+                    response);
+
+            MessageContext faultContext =
+                    MessageContextBuilder.createFaultMessageContext(msgContext, e);
+            // If the fault is not going along the back channel we should be 202ing
+            if (AddressingHelper.isFaultRedirected(this.msgContext)) {
+                response.setStatus(HttpStatus.SC_ACCEPTED);
+            } else {
+                response.sendError(HttpStatus.SC_INTERNAL_SERVER_ERROR, "Internal server error");
             }
+            engine.sendFault(faultContext);
         }
 
     }
 
-    public void handleRequest(final HttpServerConnection conn, final HttpContext context)
-            throws IOException, HttpException {
-        this.conn = conn;
-        super.handleRequest(conn, context);
-    }
+    class SimpleHTTPRequestResponseTransport implements RequestResponseTransport {
 
+        private CountDownLatch responseReadySignal = new CountDownLatch(1);
+        RequestResponseTransportStatus status = RequestResponseTransportStatus.INITIAL;
+        AxisFault faultToBeThrownOut = null;
+
+        public void acknowledgeMessage(MessageContext msgContext) throws AxisFault {
+            //TODO: Once the core HTTP API allows us to return an ack before unwinding, then the should be fixed
+            signalResponseReady();
+        }
+
+        public void awaitResponse() throws InterruptedException, AxisFault {
+            status = RequestResponseTransportStatus.WAITING;
+            responseReadySignal.await();
+
+            if (faultToBeThrownOut != null)
+                throw faultToBeThrownOut;
+        }
+
+        public void signalResponseReady() {
+            status = RequestResponseTransportStatus.SIGNALLED;
+            responseReadySignal.countDown();
+        }
+
+        public RequestResponseTransportStatus getStatus() {
+            return status;
+        }
+
+        public void signalFaultReady(AxisFault fault) {
+            faultToBeThrownOut = fault;
+            signalResponseReady();
+        }
+
+    }
+    
 }

Modified: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultConnectionListener.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultConnectionListener.java?view=diff&rev=526026&r1=526025&r2=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultConnectionListener.java (original)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultConnectionListener.java Thu Apr  5 19:12:24 2007
@@ -31,7 +31,7 @@
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.http.HttpServerConnection;
+import org.apache.http.params.HttpParams;
 
 import java.io.IOException;
 import java.net.ServerSocket;
@@ -44,40 +44,34 @@
     private volatile boolean destroyed = false;
 
     private final int port;
-    private final HttpConnectionFactory connfactory;
     private final HttpConnectionManager connmanager;
-    private ServerSocket serversocket = null;
     private final ConnectionListenerFailureHandler failureHandler;
+    private final HttpParams params;
 
-    /**
-     * Default constructor called by HttpFactory.  A custom HttpFactory subclass can call the other constructor to provide a custom ConnectionListenerErrorHandler
-     */
-    public DefaultConnectionListener(int port, HttpConnectionFactory connfactory,
-                                     HttpConnectionManager connmanager) throws IOException {
-        this(port, connfactory, connmanager, new DefaultConnectionListenerFailureHandler());
-    }
+    private ServerSocket serversocket = null;
 
     /**
      * Use this constructor to provide a custom ConnectionListenerFailureHandler, e.g. by subclassing DefaultConnectionListenerFailureHandler
      */
-    public DefaultConnectionListener(int port, HttpConnectionFactory connfactory,
-                                     HttpConnectionManager connmanager,
-                                     ConnectionListenerFailureHandler failureHandler)
-            throws IOException {
+    public DefaultConnectionListener(
+            int port,
+            final HttpConnectionManager connmanager,
+            final ConnectionListenerFailureHandler failureHandler,
+            final HttpParams params) throws IOException {
         super();
-        if (connfactory == null) {
-            throw new IllegalArgumentException("Connection factory may not be null");
-        }
         if (connmanager == null) {
             throw new IllegalArgumentException("Connection manager may not be null");
         }
         if (failureHandler == null) {
             throw new IllegalArgumentException("Failure handler may not be null");
         }
+        if (params == null) {
+            throw new IllegalArgumentException("HTTP parameters may not be null");
+        }
         this.port = port;
         this.connmanager = connmanager;
-        this.connfactory = connfactory;
         this.failureHandler = failureHandler;
+        this.params = params;
     }
 
     public void run() {
@@ -97,7 +91,7 @@
                         LOG.debug("Incoming HTTP connection from " +
                                 socket.getRemoteSocketAddress());
                     }
-                    HttpServerConnection conn = this.connfactory.newConnection(socket);
+                    AxisHttpConnection conn = new AxisHttpConnectionImpl(socket, this.params);
                     this.connmanager.process(conn);
                 } catch (Throwable ex) {
                     if (Thread.interrupted()) {

Modified: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultHttpConnectionFactory.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultHttpConnectionFactory.java?view=diff&rev=526026&r1=526025&r2=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultHttpConnectionFactory.java (original)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultHttpConnectionFactory.java Thu Apr  5 19:12:24 2007
@@ -1,103 +0,0 @@
-/*
- * $HeadURL:https://svn.apache.org/repos/asf/jakarta/httpcomponents/trunk/coyote-httpconnector/src/java/org/apache/http/tcconnector/impl/DefaultHttpConnectionFactory.java $
- * $Revision:379772 $
- * $Date:2006-02-22 14:52:29 +0100 (Wed, 22 Feb 2006) $
- *
- * ====================================================================
- *
- *  Copyright 1999-2006 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.
- * ====================================================================
- *
- * This software consists of voluntary contributions made by many
- * individuals on behalf of the Apache Software Foundation.  For more
- * information on the Apache Software Foundation, please see
- * <http://www.apache.org/>.
- *
- */
-
-package org.apache.axis2.transport.http.server;
-
-import org.apache.http.HttpServerConnection;
-import org.apache.http.impl.DefaultHttpServerConnection;
-import org.apache.http.params.HttpParams;
-
-import java.io.IOException;
-import java.net.Socket;
-
-public class DefaultHttpConnectionFactory implements HttpConnectionFactory {
-
-    final HttpParams params;
-
-    public DefaultHttpConnectionFactory(final HttpParams params) {
-        super();
-        if (params == null) {
-            throw new IllegalArgumentException("HTTP parameters may not be null");
-        }
-        this.params = params;
-    }
-
-    public HttpServerConnection newConnection(final Socket socket)
-            throws IOException {
-        DefaultHttpServerConnection conn = new Axis2HttpServerConnection();
-        conn.bind(socket, this.params);
-        return conn;
-    }
-
-    public class Axis2HttpServerConnection extends DefaultHttpServerConnection {
-        public Axis2HttpServerConnection() {
-            super();
-        }
-        //((InetAddress)socket.getLocalAddress()).getHostAddress()
-
-        public String getRemoteIPAddress() {
-            java.net.SocketAddress sa = socket.getRemoteSocketAddress();
-            if (sa instanceof java.net.InetSocketAddress) {
-                return ((java.net.InetSocketAddress) sa).getAddress().getHostAddress();
-            } else {
-                return sa.toString();
-            }
-        }
-
-         public String getLocalIPAddress() {
-            java.net.SocketAddress sa = socket.getLocalSocketAddress();
-            if (sa instanceof java.net.InetSocketAddress) {
-                return ((java.net.InetSocketAddress) sa).getAddress().getHostAddress()
-                        + ":" + socket.getLocalPort();
-            } else {
-                return sa.toString();
-            }
-        }
-
-        public String getRemoteHostName() {
-            java.net.SocketAddress sa = socket.getRemoteSocketAddress();
-            if (sa instanceof java.net.InetSocketAddress) {
-                return ((java.net.InetSocketAddress) sa).getHostName();
-            } else {
-                return sa.toString();
-                // fail-safe and fall back to something which one can use in place of the host name
-            }
-        }
-         public String getLocalHostName() {
-            java.net.SocketAddress sa = socket.getLocalSocketAddress();
-            if (sa instanceof java.net.InetSocketAddress) {
-                return ((java.net.InetSocketAddress) sa).getHostName();
-            } else {
-                return sa.toString();
-                // fail-safe and fall back to something which one can use in place of the host name
-            }
-        }
-    }
-
-}

Modified: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultHttpConnectionManager.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultHttpConnectionManager.java?view=diff&rev=526026&r1=526025&r2=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultHttpConnectionManager.java (original)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/DefaultHttpConnectionManager.java Thu Apr  5 19:12:24 2007
@@ -35,7 +35,6 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.http.ConnectionReuseStrategy;
 import org.apache.http.HttpResponseFactory;
-import org.apache.http.HttpServerConnection;
 import org.apache.http.impl.DefaultConnectionReuseStrategy;
 import org.apache.http.impl.DefaultHttpResponseFactory;
 import org.apache.http.params.HttpParams;
@@ -121,7 +120,7 @@
         this.processors.remove(processor);
     }
 
-    public void process(final HttpServerConnection conn) {
+    public void process(final AxisHttpConnection conn) {
         if (conn == null) {
             throw new IllegalArgumentException("HTTP connection may not be null");
         }
@@ -145,7 +144,7 @@
             p.addInterceptor(new ResponseContent());
             p.addInterceptor(new ResponseConnControl());
             p.addInterceptor(new ResponseSessionCookie());
-            httpProcessor = new LoggingProcessorDecorator(p);
+            httpProcessor = p;
             connStrategy = new DefaultConnectionReuseStrategy();
             responseFactory = new DefaultHttpResponseFactory();
         }

Modified: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpConnectionFactory.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpConnectionFactory.java?view=diff&rev=526026&r1=526025&r2=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpConnectionFactory.java (original)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpConnectionFactory.java Thu Apr  5 19:12:24 2007
@@ -1,41 +0,0 @@
-/*
- * $HeadURL$
- * $Revision$
- * $Date$
- *
- * ====================================================================
- *
- *  Copyright 1999-2006 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.
- * ====================================================================
- *
- * This software consists of voluntary contributions made by many
- * individuals on behalf of the Apache Software Foundation.  For more
- * information on the Apache Software Foundation, please see
- * <http://www.apache.org/>.
- *
- */
-
-package org.apache.axis2.transport.http.server;
-
-import org.apache.http.HttpServerConnection;
-
-import java.io.IOException;
-import java.net.Socket;
-
-public interface HttpConnectionFactory {
-
-    HttpServerConnection newConnection(Socket socket) throws IOException;
-
-}

Modified: webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpConnectionManager.java
URL: http://svn.apache.org/viewvc/webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpConnectionManager.java?view=diff&rev=526026&r1=526025&r2=526026
==============================================================================
--- webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpConnectionManager.java (original)
+++ webservices/axis2/branches/java/1_2/modules/kernel/src/org/apache/axis2/transport/http/server/HttpConnectionManager.java Thu Apr  5 19:12:24 2007
@@ -29,11 +29,9 @@
 
 package org.apache.axis2.transport.http.server;
 
-import org.apache.http.HttpServerConnection;
-
 public interface HttpConnectionManager {
 
-    void process(HttpServerConnection incoming);
+    void process(AxisHttpConnection incoming);
 
     void shutdown();
 



---------------------------------------------------------------------
To unsubscribe, e-mail: axis-cvs-unsubscribe@ws.apache.org
For additional commands, e-mail: axis-cvs-help@ws.apache.org