You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cxf.apache.org by ay...@apache.org on 2011/06/08 10:22:52 UTC

svn commit: r1133280 - in /cxf/trunk: distribution/src/main/release/samples/in_jvm_transport/ distribution/src/main/release/samples/in_jvm_transport/src/demo/colocated/client/ rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ rt/bindings/co...

Author: ay
Date: Wed Jun  8 08:22:51 2011
New Revision: 1133280

URL: http://svn.apache.org/viewvc?rev=1133280&view=rev
Log:
[CXF-3577] add dispatch source support for coloc

Added:
    cxf/trunk/distribution/src/main/release/samples/in_jvm_transport/src/demo/colocated/client/DispatchSourceClient.java   (with props)
Modified:
    cxf/trunk/distribution/src/main/release/samples/in_jvm_transport/pom.xml
    cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocMessageObserver.java
    cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocOutInterceptor.java
    cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocUtil.java
    cxf/trunk/rt/bindings/coloc/src/test/java/org/apache/cxf/binding/coloc/ColocOutInterceptorTest.java
    cxf/trunk/rt/frontend/jaxws/src/main/java/org/apache/cxf/jaxws/DispatchImpl.java

Modified: cxf/trunk/distribution/src/main/release/samples/in_jvm_transport/pom.xml
URL: http://svn.apache.org/viewvc/cxf/trunk/distribution/src/main/release/samples/in_jvm_transport/pom.xml?rev=1133280&r1=1133279&r2=1133280&view=diff
==============================================================================
--- cxf/trunk/distribution/src/main/release/samples/in_jvm_transport/pom.xml (original)
+++ cxf/trunk/distribution/src/main/release/samples/in_jvm_transport/pom.xml Wed Jun  8 08:22:51 2011
@@ -110,5 +110,10 @@
             <artifactId>cxf-rt-transports-http-jetty</artifactId>
             <version>2.4.1-SNAPSHOT</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.cxf</groupId>
+            <artifactId>cxf-rt-bindings-coloc</artifactId>
+            <version>2.4.1-SNAPSHOT</version>
+        </dependency>
     </dependencies>
 </project>

Added: cxf/trunk/distribution/src/main/release/samples/in_jvm_transport/src/demo/colocated/client/DispatchSourceClient.java
URL: http://svn.apache.org/viewvc/cxf/trunk/distribution/src/main/release/samples/in_jvm_transport/src/demo/colocated/client/DispatchSourceClient.java?rev=1133280&view=auto
==============================================================================
--- cxf/trunk/distribution/src/main/release/samples/in_jvm_transport/src/demo/colocated/client/DispatchSourceClient.java (added)
+++ cxf/trunk/distribution/src/main/release/samples/in_jvm_transport/src/demo/colocated/client/DispatchSourceClient.java Wed Jun  8 08:22:51 2011
@@ -0,0 +1,183 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you 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 demo.colocated.client;
+
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import javax.xml.namespace.QName;
+import javax.xml.transform.Source;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.sax.SAXResult;
+import javax.xml.transform.stream.StreamSource;
+import javax.xml.ws.Dispatch;
+import javax.xml.ws.Service;
+import javax.xml.ws.handler.MessageContext;
+import javax.xml.ws.soap.SOAPBinding;
+import javax.xml.ws.soap.SOAPFaultException;
+
+import org.xml.sax.Attributes;
+import org.xml.sax.SAXException;
+import org.xml.sax.helpers.DefaultHandler;
+
+import org.apache.cxf.Bus;
+import org.apache.cxf.BusFactory;
+import org.apache.cxf.endpoint.Client;
+import org.apache.cxf.jaxws.DispatchImpl;
+import org.apache.cxf.service.model.BindingInfo;
+import org.apache.cxf.service.model.BindingOperationInfo;
+import org.apache.cxf.service.model.InterfaceInfo;
+import org.apache.cxf.service.model.OperationInfo;
+import org.apache.hello_world_soap_http.Greeter;
+import org.apache.hello_world_soap_http.PingMeFault;
+import org.apache.hello_world_soap_http.SOAPService;
+import org.apache.hello_world_soap_http.types.FaultDetail;
+
+import demo.colocated.server.Server;
+
+public final class DispatchSourceClient {
+    private static final String SERVICE_NS = "http://apache.org/hello_world_soap_http"; 
+    private static final QName SERVICE_NAME = new QName(SERVICE_NS, "SOAPService");
+    private static final QName PORT_NAME = new QName(SERVICE_NS, "SoapPort");
+    private static final String PAYLOAD_NAMESPACE_URI = "http://apache.org/hello_world_soap_http/types";
+
+    private static final String SAYHI_REQUEST_TEMPLATE = "<ns1:sayHi xmlns:ns1=\"http://apache.org/hello_world_soap_http/types\" />";
+    private static final String GREETME_REQUEST_TEMPLATE = "<ns1:greetMe xmlns:ns1=\"http://apache.org/hello_world_soap_http/types\"><ns1:requestType>%s</ns1:requestType></ns1:greetMe>";
+    private static final String PINGME_REQUEST_TEMPLATE = "<ns1:pingMe xmlns:ns1=\"http://apache.org/hello_world_soap_http/types\" />";
+
+    private static final QName SAYHI_OPERATION_NAME = new QName(SERVICE_NS, "sayHi");
+    private static final QName GREETME_OPERATION_NAME = new QName(SERVICE_NS, "greetMe");
+    private static final QName PINGME_OPERATION_NAME = new QName(SERVICE_NS, "pingMe");
+        
+    final static String ADDRESS = "http://localhost:9000/SoapContext/GreeterPort";
+
+
+    private DispatchSourceClient() {
+    }
+
+    public static void main(String args[]) throws Exception {
+
+        Server.main(new String[]{"inProcess"});
+        
+        Service service = Service.create(SERVICE_NAME);
+        service.addPort(PORT_NAME, SOAPBinding.SOAP11HTTP_BINDING, ADDRESS);
+        
+        Dispatch<Source> dispatch = service.createDispatch(PORT_NAME, Source.class, Service.Mode.PAYLOAD);
+        
+        String resp;
+        Source response;
+        
+        System.out.println("Invoking sayHi...");
+        setOperation(dispatch, SAYHI_OPERATION_NAME);
+        response = dispatch.invoke(encodeSource(SAYHI_REQUEST_TEMPLATE, null));
+        resp = decodeSource(response, PAYLOAD_NAMESPACE_URI, "responseType");
+        System.out.println("Server responded with: " + resp);
+        System.out.println();
+
+        System.out.println("Invoking greetMe...");
+        setOperation(dispatch, GREETME_OPERATION_NAME);
+        response = dispatch.invoke(encodeSource(GREETME_REQUEST_TEMPLATE, System.getProperty("user.name")));
+        resp = decodeSource(response, PAYLOAD_NAMESPACE_URI, "responseType");
+        System.out.println("Server responded with: " + resp);
+        System.out.println();
+
+        try {
+            System.out.println("Invoking pingMe, expecting exception...");
+            setOperation(dispatch, PINGME_OPERATION_NAME);
+            response = dispatch.invoke(encodeSource(PINGME_REQUEST_TEMPLATE, null));
+        } catch (SOAPFaultException ex) {
+            System.out.println("Expected exception: SoapFault has occurred: " + ex.getMessage());
+        }
+        System.exit(0);
+    }
+    
+    private static void setOperation(Dispatch<Source> dispatch, QName operationName) {
+        dispatch.getRequestContext().put(MessageContext.WSDL_OPERATION, operationName);        
+    }
+
+    private static Source encodeSource(String template, String value) throws IOException {
+        String payload = value == null ? template : String.format(template, value);
+        Source source = new StreamSource(new ByteArrayInputStream(payload.getBytes("utf-8")));
+        return source;
+    }
+
+    private static String decodeSource(Source source, String uri, String name) throws Exception {
+        Transformer transformer = TransformerFactory.newInstance().newTransformer();
+        ContentHandler handler = new ContentHandler(uri, name);
+        transformer.transform(source, new SAXResult(handler));
+        return handler.getValue();
+    }
+    
+    static class ContentHandler extends DefaultHandler {
+        StringBuffer buffer;
+        String namespaceURI;
+        String elementName;
+        boolean recording;
+        
+        public ContentHandler(String namespaceURI, String elementName) {
+            this.namespaceURI = namespaceURI;
+            this.elementName = elementName;
+        }
+        
+        /* (non-Javadoc)
+         * @see org.xml.sax.helpers.DefaultHandler#startElement(java.lang.String, java.lang.String, java.lang.String, org.xml.sax.Attributes)
+         */
+        @Override
+        public void startElement(String uri, String localName, String qName, Attributes attributes)
+            throws SAXException {
+            if (namespaceURI.equals(uri) && elementName.equals(localName)) {
+                recording = true;
+            }
+        }
+
+
+        /* (non-Javadoc)
+         * @see org.xml.sax.helpers.DefaultHandler#endElement(java.lang.String, java.lang.String, java.lang.String)
+         */
+        @Override
+        public void endElement(String uri, String localName, String qName) throws SAXException {
+            if (namespaceURI.equals(uri) && elementName.equals(localName)) {
+                recording = false;
+            }
+        }
+
+        /* (non-Javadoc)
+         * @see org.xml.sax.helpers.DefaultHandler#characters(char[], int, int)
+         */
+        @Override
+        public void characters(char[] ch, int start, int length) throws SAXException {
+            if (recording) {
+                if (buffer == null) {
+                    buffer = new StringBuffer();
+                }
+                buffer.append(new String(ch, start, length));
+            }
+        }
+
+        /**
+         * @return
+         */
+        public String getValue() {
+            return buffer == null ? null : buffer.toString();
+        }
+    }    
+}

Propchange: cxf/trunk/distribution/src/main/release/samples/in_jvm_transport/src/demo/colocated/client/DispatchSourceClient.java
------------------------------------------------------------------------------
    svn:executable = *

Modified: cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocMessageObserver.java
URL: http://svn.apache.org/viewvc/cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocMessageObserver.java?rev=1133280&r1=1133279&r2=1133280&view=diff
==============================================================================
--- cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocMessageObserver.java (original)
+++ cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocMessageObserver.java Wed Jun  8 08:22:51 2011
@@ -26,6 +26,7 @@ import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import javax.xml.namespace.QName;
+import javax.xml.transform.Source;
 
 import org.apache.cxf.Bus;
 import org.apache.cxf.BusFactory;
@@ -50,7 +51,7 @@ import org.apache.cxf.transport.ChainIni
 public class ColocMessageObserver extends ChainInitiationObserver {
     private static final Logger LOG = LogUtils.getL7dLogger(ColocMessageObserver.class);
     private static final String COLOCATED = Message.class.getName() + ".COLOCATED";
-    
+
     public ColocMessageObserver(Endpoint endpoint, Bus bus) {
         super(endpoint, bus);
     }
@@ -98,8 +99,33 @@ public class ColocMessageObserver extend
             chain.add(addColocInterceptors());
             inMsg.setInterceptorChain(chain);
     
+            //Convert the coloc object type if necessary
+            OperationInfo soi = m.getExchange().get(OperationInfo.class);
+            if (soi != null && oi != null) {
+                if (ColocUtil.isAssignableOperationInfo(soi, Source.class) 
+                    && !ColocUtil.isAssignableOperationInfo(oi, Source.class)) {
+                    // converting source -> pojo
+                    ColocUtil.convertSourceToObject(inMsg);
+                } else if (ColocUtil.isAssignableOperationInfo(oi, Source.class) 
+                    && !ColocUtil.isAssignableOperationInfo(soi, Source.class)) {
+                    // converting pojo -> source
+                    ColocUtil.convertObjectToSource(inMsg);
+                }
+            }
             chain.doIntercept(inMsg);
-    
+            if (soi != null && oi != null) {
+                if (ColocUtil.isAssignableOperationInfo(soi, Source.class) 
+                    && !ColocUtil.isAssignableOperationInfo(oi, Source.class)
+                    && ex.getOutMessage() != null) {
+                    // converting pojo -> source                
+                    ColocUtil.convertObjectToSource(ex.getOutMessage());
+                } else if (ColocUtil.isAssignableOperationInfo(oi, Source.class) 
+                    && !ColocUtil.isAssignableOperationInfo(soi, Source.class)
+                    && ex.getOutMessage() != null) {
+                    // converting pojo -> source
+                    ColocUtil.convertSourceToObject(ex.getOutMessage());
+                }
+            }
             //Set Server OutBound Message onto InBound Exchange.
             setOutBoundMessage(ex, m.getExchange());
         } finally {

Modified: cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocOutInterceptor.java
URL: http://svn.apache.org/viewvc/cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocOutInterceptor.java?rev=1133280&r1=1133279&r2=1133280&view=diff
==============================================================================
--- cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocOutInterceptor.java (original)
+++ cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocOutInterceptor.java Wed Jun  8 08:22:51 2011
@@ -192,7 +192,7 @@ public class ColocOutInterceptor extends
                     //Check For Operation Match.
                     BindingOperationInfo receiverOI = receiverEI.getBinding().getOperation(boi.getName());
                     if (receiverOI != null 
-                        && isSameOperationInfo(boi, receiverOI)) {
+                        && isCompatibleOperationInfo(boi, receiverOI)) {
                         return s;
                     }
                 }
@@ -207,7 +207,13 @@ public class ColocOutInterceptor extends
         return ColocUtil.isSameOperationInfo(sender.getOperationInfo(), 
                                              receiver.getOperationInfo());
     }
-    
+
+    protected boolean isCompatibleOperationInfo(BindingOperationInfo sender,
+                                                BindingOperationInfo receiver) {
+        return ColocUtil.isCompatibleOperationInfo(sender.getOperationInfo(), 
+                                                   receiver.getOperationInfo());
+    }
+
     public void setExchangeProperties(Exchange exchange, Endpoint ep) {
         exchange.put(Endpoint.class, ep);
         exchange.put(Service.class, ep.getService());

Modified: cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocUtil.java
URL: http://svn.apache.org/viewvc/cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocUtil.java?rev=1133280&r1=1133279&r2=1133280&view=diff
==============================================================================
--- cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocUtil.java (original)
+++ cxf/trunk/rt/bindings/coloc/src/main/java/org/apache/cxf/binding/coloc/ColocUtil.java Wed Jun  8 08:22:51 2011
@@ -18,6 +18,9 @@
  */
 package org.apache.cxf.binding.coloc;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.OutputStream;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
@@ -25,8 +28,15 @@ import java.util.SortedSet;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import javax.xml.soap.SOAPMessage;
+import javax.xml.stream.XMLStreamReader;
+import javax.xml.transform.Source;
+import javax.xml.transform.stream.StreamSource;
+
 import org.apache.cxf.Bus;
 import org.apache.cxf.common.logging.LogUtils;
+import org.apache.cxf.databinding.DataReader;
+import org.apache.cxf.databinding.DataWriter;
 import org.apache.cxf.endpoint.Endpoint;
 import org.apache.cxf.helpers.CastUtils;
 import org.apache.cxf.interceptor.Interceptor;
@@ -34,12 +44,15 @@ import org.apache.cxf.interceptor.Interc
 import org.apache.cxf.interceptor.InterceptorProvider;
 import org.apache.cxf.message.Exchange;
 import org.apache.cxf.message.Message;
+import org.apache.cxf.message.MessageContentsList;
+import org.apache.cxf.message.MessageUtils;
 import org.apache.cxf.phase.Phase;
 import org.apache.cxf.phase.PhaseInterceptorChain;
 import org.apache.cxf.service.model.FaultInfo;
 import org.apache.cxf.service.model.MessageInfo;
 import org.apache.cxf.service.model.MessagePartInfo;
 import org.apache.cxf.service.model.OperationInfo;
+import org.apache.cxf.staxutils.StaxUtils;
 
 public final class ColocUtil {
     private static final Logger LOG = LogUtils.getL7dLogger(ColocUtil.class);
@@ -165,6 +178,20 @@ public final class ColocUtil {
                 && isSameFaultInfo(oi1.getFaults(), oi2.getFaults());
     }
     
+    public static boolean isCompatibleOperationInfo(OperationInfo oi1, OperationInfo oi2) {
+        return isSameOperationInfo(oi1, oi2)
+               || isAssignableOperationInfo(oi1, Source.class) 
+               || isAssignableOperationInfo(oi2, Source.class)
+               || isAssignableOperationInfo(oi1, SOAPMessage.class) 
+               || isAssignableOperationInfo(oi2, SOAPMessage.class);
+    }
+    
+    public static boolean isAssignableOperationInfo(OperationInfo oi, Class<?> cls) {
+        MessageInfo mi = oi.getInput();
+        List<MessagePartInfo> mpis = mi.getMessageParts();
+        return mpis.size() == 1 && cls.isAssignableFrom(mpis.get(0).getTypeClass());
+    }
+    
     public static boolean isSameMessageInfo(MessageInfo mi1, MessageInfo mi2) {
         if ((mi1 == null && mi2 != null)
             || (mi1 != null && mi2 == null)) {
@@ -223,4 +250,48 @@ public final class ColocUtil {
         }
         return true;
     }
+    
+    public static void convertSourceToObject(Message message) {
+        List<Object> content = CastUtils.cast(message.getContent(List.class));
+        if (content == null || content.size() < 1) {
+            // nothing to convert
+            return;
+        }
+        // only supporting the wrapped style for now  (one pojo <-> one source)
+        Source source = (Source)content.get(0);
+        DataReader<XMLStreamReader> reader =
+            message.getExchange().getService().getDataBinding().createReader(XMLStreamReader.class);
+        MessagePartInfo mpi = getMessageInfo(message).getMessagePart(0);
+        Object wrappedObject = reader.read(mpi, StaxUtils.createXMLStreamReader(source));
+        MessageContentsList parameters = new MessageContentsList();
+        parameters.put(mpi, wrappedObject);
+
+        message.setContent(List.class, parameters);
+    }
+
+    public static void convertObjectToSource(Message message) {
+        List<Object> content = CastUtils.cast(message.getContent(List.class));
+        if (content == null || content.size() < 1) {
+            // nothing to convert
+            return;
+        }
+        // only supporting the wrapped style for now  (one pojo <-> one source)
+        Object object = content.get(0);
+        DataWriter<OutputStream> writer = 
+            message.getExchange().getService().getDataBinding().createWriter(OutputStream.class);
+        //TODO use a better conversion method to get a Source from a pojo.
+        ByteArrayOutputStream bos = new ByteArrayOutputStream();
+        writer.write(object, bos);
+
+        content.set(0, new StreamSource(new ByteArrayInputStream(bos.toByteArray())));
+    }
+    
+    private static MessageInfo getMessageInfo(Message message) {
+        OperationInfo oi = message.getExchange().get(OperationInfo.class);
+        if (MessageUtils.isOutbound(message)) {
+            return oi.getOutput();
+        } else {
+            return oi.getInput();
+        }
+    }
 }

Modified: cxf/trunk/rt/bindings/coloc/src/test/java/org/apache/cxf/binding/coloc/ColocOutInterceptorTest.java
URL: http://svn.apache.org/viewvc/cxf/trunk/rt/bindings/coloc/src/test/java/org/apache/cxf/binding/coloc/ColocOutInterceptorTest.java?rev=1133280&r1=1133279&r2=1133280&view=diff
==============================================================================
--- cxf/trunk/rt/bindings/coloc/src/test/java/org/apache/cxf/binding/coloc/ColocOutInterceptorTest.java (original)
+++ cxf/trunk/rt/bindings/coloc/src/test/java/org/apache/cxf/binding/coloc/ColocOutInterceptorTest.java Wed Jun  8 08:22:51 2011
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import javax.xml.namespace.QName;
+import javax.xml.transform.Source;
 
 import org.apache.cxf.Bus;
 import org.apache.cxf.BusFactory;
@@ -45,6 +46,9 @@ import org.apache.cxf.service.model.Bind
 import org.apache.cxf.service.model.BindingOperationInfo;
 import org.apache.cxf.service.model.EndpointInfo;
 import org.apache.cxf.service.model.InterfaceInfo;
+import org.apache.cxf.service.model.MessageInfo;
+import org.apache.cxf.service.model.MessagePartInfo;
+import org.apache.cxf.service.model.OperationInfo;
 import org.apache.cxf.service.model.ServiceInfo;
 import org.easymock.classextension.EasyMock;
 import org.easymock.classextension.IMocksControl;
@@ -153,6 +157,7 @@ public class ColocOutInterceptorTest ext
         verifyIsColocatedWithDifferentEndpoint();
         verifyIsColocatedWithDifferentOperation();
         verifyIsColocatedWithSameOperation();
+        verifyIsColocatedWithCompatibleOperation();
     }
 
     @Test
@@ -169,9 +174,19 @@ public class ColocOutInterceptorTest ext
         list.add(s1);        
         Endpoint sep = control.createMock(Endpoint.class);
         ex.put(Endpoint.class, sep);
+        QName op = new QName("E", "F");
+        QName intf = new QName("G", "H");
         BindingInfo sbi = control.createMock(BindingInfo.class);
-        InterfaceInfo sii = control.createMock(InterfaceInfo.class);
+        ServiceInfo ssi = new ServiceInfo();
+        InterfaceInfo sii = new InterfaceInfo(ssi, intf);
+        sii.addOperation(op);
+        OperationInfo soi = sii.getOperation(op);
+        ServiceInfo rsi = new ServiceInfo();
+        InterfaceInfo rii = new InterfaceInfo(rsi, intf);
+        rii.addOperation(op);
+        OperationInfo roi = rii.getOperation(op);
         BindingOperationInfo sboi = control.createMock(BindingOperationInfo.class);
+        BindingOperationInfo rboi = control.createMock(BindingOperationInfo.class);
         
         ex.put(BindingOperationInfo.class, sboi);
         //Local var
@@ -182,7 +197,7 @@ public class ColocOutInterceptorTest ext
         Service res = control.createMock(Service.class);
         BindingInfo rbi = control.createMock(BindingInfo.class);
         EndpointInfo rei = control.createMock(EndpointInfo.class);
-
+        
         EasyMock.expect(sr.getServers()).andReturn(list);
         EasyMock.expect(sep.getService()).andReturn(ses);
         EasyMock.expect(sep.getEndpointInfo()).andReturn(sei);
@@ -195,16 +210,16 @@ public class ColocOutInterceptorTest ext
         EasyMock.expect(sei.getName()).andReturn(new QName("C", "D"));
         EasyMock.expect(rei.getBinding()).andReturn(rbi);
 
-        QName intf = new QName("G", "H");
-        QName op = new QName("E", "F");
         EasyMock.expect(sboi.getName()).andReturn(op).anyTimes();
-        EasyMock.expect(rbi.getOperation(op)).andReturn(sboi);
-        
+        EasyMock.expect(sboi.getOperationInfo()).andReturn(soi);
+        EasyMock.expect(rboi.getName()).andReturn(op).anyTimes();
+        EasyMock.expect(rboi.getOperationInfo()).andReturn(roi);
+        EasyMock.expect(rbi.getOperation(op)).andReturn(rboi);
+
         InterceptorChain chain = control.createMock(InterceptorChain.class);
         msg.setInterceptorChain(chain);
         EasyMock.expect(sboi.getBinding()).andReturn(sbi);
         EasyMock.expect(sbi.getInterface()).andReturn(sii);
-        EasyMock.expect(sii.getName()).andReturn(intf);
         
         control.replay();
         colocOut.handleMessage(msg);
@@ -394,6 +409,7 @@ public class ColocOutInterceptorTest ext
         list.add(s1);        
         Endpoint sep = control.createMock(Endpoint.class);
         BindingOperationInfo sboi = control.createMock(BindingOperationInfo.class);
+
         //Local var
         Service ses = control.createMock(Service.class);
         EndpointInfo sei = control.createMock(EndpointInfo.class);
@@ -401,7 +417,28 @@ public class ColocOutInterceptorTest ext
         Endpoint rep = control.createMock(Endpoint.class);
         Service res = control.createMock(Service.class);
         EndpointInfo rei = control.createMock(EndpointInfo.class);
-
+        BindingOperationInfo rboi = control.createMock(BindingOperationInfo.class);
+        
+        QName op = new QName("E", "F");
+        QName intf = new QName("G", "H");
+        QName inmi = new QName("M", "in");
+        QName outmi = new QName("M", "out");
+        ServiceInfo ssi = new ServiceInfo();
+        InterfaceInfo sii = new InterfaceInfo(ssi, intf);
+        sii.addOperation(op);
+        OperationInfo soi = sii.getOperation(op);
+        MessageInfo mii = new MessageInfo(soi, MessageInfo.Type.INPUT, inmi);
+        MessageInfo mio = new MessageInfo(soi, MessageInfo.Type.OUTPUT, outmi);
+        soi.setInput("in", mii);
+        soi.setOutput("out", mio);
+        
+        ServiceInfo rsi = new ServiceInfo();
+        InterfaceInfo rii = new InterfaceInfo(rsi, intf);
+        rii.addOperation(op);
+        OperationInfo roi = rii.getOperation(op);
+        roi.setInput("in", mii);
+        roi.setOutput("out", mio);
+        
         EasyMock.expect(sep.getService()).andReturn(ses);
         EasyMock.expect(sep.getEndpointInfo()).andReturn(sei);
         EasyMock.expect(s1.getEndpoint()).andReturn(rep);
@@ -412,9 +449,85 @@ public class ColocOutInterceptorTest ext
         EasyMock.expect(rei.getName()).andReturn(new QName("C", "D"));
         EasyMock.expect(sei.getName()).andReturn(new QName("C", "D"));
         EasyMock.expect(rei.getBinding()).andReturn(rbi);
+        
+        EasyMock.expect(sboi.getName()).andReturn(op);
+        EasyMock.expect(sboi.getOperationInfo()).andReturn(soi);
+        EasyMock.expect(rboi.getName()).andReturn(op);
+        EasyMock.expect(rboi.getOperationInfo()).andReturn(roi);
+        EasyMock.expect(rbi.getOperation(op)).andReturn(rboi);
+        
+        control.replay();
+        Server val = colocOut.isColocated(list, sep, sboi);
+        assertEquals("Expecting a colocated call",
+                     s1,
+                     val);
+        control.reset();
+    }
+
+    private void verifyIsColocatedWithCompatibleOperation() {
+        colocOut = new TestColocOutInterceptor1();
+        //Funtion Param
+        Server s1 = control.createMock(Server.class);
+        List<Server> list = new ArrayList<Server>();
+        list.add(s1);        
+        Endpoint sep = control.createMock(Endpoint.class);
+        BindingOperationInfo sboi = control.createMock(BindingOperationInfo.class);
+
+        //Local var
+        Service ses = control.createMock(Service.class);
+        EndpointInfo sei = control.createMock(EndpointInfo.class);
+        BindingInfo rbi = control.createMock(BindingInfo.class);
+        Endpoint rep = control.createMock(Endpoint.class);
+        Service res = control.createMock(Service.class);
+        EndpointInfo rei = control.createMock(EndpointInfo.class);
+        BindingOperationInfo rboi = control.createMock(BindingOperationInfo.class);
+        
         QName op = new QName("E", "F");
+        QName intf = new QName("G", "H");
+        QName inmi = new QName("M", "in");
+        QName outmi = new QName("M", "out");
+        ServiceInfo ssi = new ServiceInfo();
+        InterfaceInfo sii = new InterfaceInfo(ssi, intf);
+        sii.addOperation(op);
+        OperationInfo soi = sii.getOperation(op);
+        MessageInfo mii = new MessageInfo(soi, MessageInfo.Type.INPUT, inmi);
+        MessagePartInfo mpi = mii.addMessagePart("parameters");
+        mpi.setTypeClass(Source.class);
+        MessageInfo mio = new MessageInfo(soi, MessageInfo.Type.OUTPUT, outmi);
+        mpi = mio.addMessagePart("parameters");
+        mpi.setTypeClass(Source.class);
+        soi.setInput("in", mii);
+        soi.setOutput("out", mio);
+        
+        ServiceInfo rsi = new ServiceInfo();
+        InterfaceInfo rii = new InterfaceInfo(rsi, intf);
+        rii.addOperation(op);
+        OperationInfo roi = rii.getOperation(op);
+        mii = new MessageInfo(roi, MessageInfo.Type.INPUT, inmi);
+        mpi = mii.addMessagePart("parameters");
+        mpi.setTypeClass(Object.class);
+        mio = new MessageInfo(roi, MessageInfo.Type.OUTPUT, outmi);
+        mpi = mio.addMessagePart("parameters");
+        mpi.setTypeClass(Object.class);
+        roi.setInput("in", mii);
+        roi.setOutput("out", mio);
+        
+        EasyMock.expect(sep.getService()).andReturn(ses);
+        EasyMock.expect(sep.getEndpointInfo()).andReturn(sei);
+        EasyMock.expect(s1.getEndpoint()).andReturn(rep);
+        EasyMock.expect(rep.getService()).andReturn(res);
+        EasyMock.expect(rep.getEndpointInfo()).andReturn(rei);
+        EasyMock.expect(ses.getName()).andReturn(new QName("A", "B"));
+        EasyMock.expect(res.getName()).andReturn(new QName("A", "B"));
+        EasyMock.expect(rei.getName()).andReturn(new QName("C", "D"));
+        EasyMock.expect(sei.getName()).andReturn(new QName("C", "D"));
+        EasyMock.expect(rei.getBinding()).andReturn(rbi);
+        
         EasyMock.expect(sboi.getName()).andReturn(op);
-        EasyMock.expect(rbi.getOperation(op)).andReturn(sboi);
+        EasyMock.expect(sboi.getOperationInfo()).andReturn(soi);
+        EasyMock.expect(rboi.getName()).andReturn(op);
+        EasyMock.expect(rboi.getOperationInfo()).andReturn(roi);
+        EasyMock.expect(rbi.getOperation(op)).andReturn(rboi);
         
         control.replay();
         Server val = colocOut.isColocated(list, sep, sboi);
@@ -439,9 +552,6 @@ public class ColocOutInterceptorTest ext
             //No Op
         }
         
-        protected boolean isSameOperationInfo(BindingOperationInfo s, BindingOperationInfo r) {
-            return true;
-        }
     }
     
     class TestBindingInfo extends BindingInfo {

Modified: cxf/trunk/rt/frontend/jaxws/src/main/java/org/apache/cxf/jaxws/DispatchImpl.java
URL: http://svn.apache.org/viewvc/cxf/trunk/rt/frontend/jaxws/src/main/java/org/apache/cxf/jaxws/DispatchImpl.java?rev=1133280&r1=1133279&r2=1133280&view=diff
==============================================================================
--- cxf/trunk/rt/frontend/jaxws/src/main/java/org/apache/cxf/jaxws/DispatchImpl.java (original)
+++ cxf/trunk/rt/frontend/jaxws/src/main/java/org/apache/cxf/jaxws/DispatchImpl.java Wed Jun  8 08:22:51 2011
@@ -92,6 +92,8 @@ public class DispatchImpl<T> implements 
     private static final String DISPATCH_NS = "http://cxf.apache.org/jaxws/dispatch";
     private static final String INVOKE_NAME = "Invoke";
     private static final String INVOKE_ONEWAY_NAME = "InvokeOneWay";
+    private static final QName INVOKE_QNAME = new QName(DISPATCH_NS, INVOKE_NAME);
+    private static final QName INVOKE_ONEWAY_QNAME = new QName(DISPATCH_NS, INVOKE_ONEWAY_NAME);
     
     private final Binding binding;
     private final EndpointReferenceBuilder builder;
@@ -157,7 +159,7 @@ public class DispatchImpl<T> implements 
             
         ServiceInfo info = client.getEndpoint().getEndpointInfo().getService();
         OperationInfo opInfo = info.getInterface()
-            .addOperation(new QName(DISPATCH_NS, name));
+            .addOperation(oneWay ? INVOKE_ONEWAY_QNAME : INVOKE_QNAME);
         MessageInfo mInfo = opInfo.createMessage(new QName(DISPATCH_NS, name + "Request"), Type.INPUT);
         opInfo.setInput(name + "Request", mInfo);
         MessagePartInfo mpi = mInfo.addMessagePart("parameters");
@@ -182,6 +184,25 @@ public class DispatchImpl<T> implements 
         }
     }
     
+    private void addInvokeOperation(QName operationName, boolean oneWay) {
+        ServiceInfo info = client.getEndpoint().getEndpointInfo().getService();
+        
+        OperationInfo invokeOpInfo = info.getInterface()
+                       .getOperation(oneWay ? INVOKE_ONEWAY_QNAME : INVOKE_QNAME);
+        
+        OperationInfo opInfo = info.getInterface().addOperation(operationName);
+        opInfo.setInput(invokeOpInfo.getInputName(), invokeOpInfo.getInput());
+
+        if (!oneWay) {
+            opInfo.setOutput(invokeOpInfo.getOutputName(), invokeOpInfo.getOutput());
+        }
+        
+        for (BindingInfo bind : client.getEndpoint().getEndpointInfo().getService().getBindings()) {
+            BindingOperationInfo bo = new BindingOperationInfo(bind, opInfo);
+            bind.addOperation(bo);
+        }
+    }
+    
     public Map<String, Object> getRequestContext() {
         return new WrappedMessageContext(client.getRequestContext(),
                                          null,
@@ -283,8 +304,13 @@ public class DispatchImpl<T> implements 
             QName opName = (QName)getRequestContext().get(MessageContext.WSDL_OPERATION);
                        
             if (opName == null) {
-                opName = new QName(DISPATCH_NS,
-                                   isOneWay ? INVOKE_ONEWAY_NAME : INVOKE_NAME);
+                opName = isOneWay ? INVOKE_ONEWAY_QNAME : INVOKE_QNAME;
+            } else {
+                BindingOperationInfo bop = client.getEndpoint().getBinding()
+                                            .getBindingInfo().getOperation(opName);
+                if (bop == null) {
+                    addInvokeOperation(opName, isOneWay);
+                }
             }
             
             //CXF-2836 : find the operation for the dispatched object 
@@ -351,7 +377,13 @@ public class DispatchImpl<T> implements 
         try {
             QName opName = (QName)getRequestContext().get(MessageContext.WSDL_OPERATION);
             if (opName == null) {
-                opName = new QName(DISPATCH_NS, INVOKE_NAME);
+                opName = INVOKE_QNAME;
+            } else {
+                BindingOperationInfo bop = client.getEndpoint().getBinding()
+                    .getBindingInfo().getOperation(opName);
+                if (bop == null) {
+                    addInvokeOperation(opName, false);
+                }
             }
 
             client.invokeWrapped(callback,