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 2006/12/15 16:08:06 UTC

svn commit: r487578 - in /webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2: client/ description/

Author: dims
Date: Fri Dec 15 07:08:05 2006
New Revision: 487578

URL: http://svn.apache.org/viewvc?view=rev&rev=487578
Log:
Fix for AXIS2-1841 - Duplicate code in OperationClient

Added:
    webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/TwoChannelAxisOperation.java
Modified:
    webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/client/OperationClient.java
    webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/InOutAxisOperation.java
    webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/OutInAxisOperation.java
    webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/OutOnlyAxisOperation.java

Modified: webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/client/OperationClient.java
URL: http://svn.apache.org/viewvc/webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/client/OperationClient.java?view=diff&rev=487578&r1=487577&r2=487578
==============================================================================
--- webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/client/OperationClient.java (original)
+++ webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/client/OperationClient.java Fri Dec 15 07:08:05 2006
@@ -16,10 +16,24 @@
 
 package org.apache.axis2.client;
 
+import org.apache.axiom.om.OMElement;
 import org.apache.axis2.AxisFault;
+import org.apache.axis2.addressing.EndpointReference;
 import org.apache.axis2.client.async.Callback;
+import org.apache.axis2.context.ConfigurationContext;
 import org.apache.axis2.context.MessageContext;
 import org.apache.axis2.context.OperationContext;
+import org.apache.axis2.context.ServiceContext;
+import org.apache.axis2.description.AxisOperation;
+import org.apache.axis2.description.TransportOutDescription;
+import org.apache.axis2.description.ClientUtils;
+import org.apache.axis2.i18n.Messages;
+import org.apache.axis2.util.UUIDGenerator;
+import org.apache.axis2.util.TargetResolver;
+import org.apache.axis2.wsdl.WSDLConstants;
+
+import java.util.Iterator;
+import java.util.Map;
 
 /**
  * An operation client is the way an advanced user interacts with Axis2. Actual
@@ -31,7 +45,34 @@
  * in a new message being created, then if a message receiver is registered with
  * the client then the message will be delivered to that client.
  */
-public interface  OperationClient {
+public abstract class OperationClient {
+
+     protected AxisOperation axisOp;
+ 
+     protected ServiceContext sc;
+ 
+     protected Options options;
+ 
+     protected OperationContext oc;
+ 
+     protected Callback callback;
+ 
+     /*
+      * indicates whether the MEP execution has completed (and hence ready for
+      * resetting)
+      */
+     protected boolean completed;
+ 
+     protected OperationClient(AxisOperation axisOp, ServiceContext sc,
+                               Options options) {
+         this.axisOp = axisOp;
+         this.sc = sc;
+         this.options = options;
+         this.completed = false;
+         this.oc = new OperationContext(axisOp);
+         this.oc.setParent(this.sc);
+     }
+
     /**
      * Sets the options that should be used for this particular client. This
      * resets the entire set of options to use the new options - so you'd lose
@@ -39,7 +80,9 @@
      *
      * @param options the options
      */
-    public void setOptions(Options options);
+    public void setOptions(Options options) {
+        this.options = options;
+    }
 
     /**
      * Return the options used by this client. If you want to set a single
@@ -48,7 +91,9 @@
      *
      * @return the options, which will never be null.
      */
-    public Options getOptions();
+    public Options getOptions() {
+        return options;
+    }
 
     /**
      * Add a message context to the client for processing. This method must not
@@ -58,7 +103,7 @@
      * @param messageContext the message context
      * @throws AxisFault if this is called inappropriately.
      */
-    public void addMessageContext(MessageContext messageContext) throws AxisFault;
+    public abstract void addMessageContext(MessageContext messageContext) throws AxisFault;
 
     /**
      * Return a message from the client - will return null if the requested
@@ -68,7 +113,7 @@
      * @return the desired message context or null if its not available.
      * @throws AxisFault if the message label is invalid
      */
-    public MessageContext getMessageContext(String messageLabel)
+    public abstract MessageContext getMessageContext(String messageLabel)
             throws AxisFault;
 
     /**
@@ -81,7 +126,7 @@
      * @param callback the callback to be used when the client decides its time to
      *                 use it
      */
-    public void setCallback(Callback callback);
+    public abstract void setCallback(Callback callback);
 
     /**
      * Execute the MEP. What this does depends on the specific operation client.
@@ -97,7 +142,7 @@
      * @throws AxisFault if something goes wrong during the execution of the operation
      *                   client.
      */
-    public void execute(boolean block) throws AxisFault;
+    public abstract void execute(boolean block) throws AxisFault;
 
     /**
      * Reset the operation client to a clean status after the MEP has completed.
@@ -107,7 +152,14 @@
      * @throws AxisFault if reset is called before the MEP client has completed an
      *                   interaction.
      */
-    public void reset() throws AxisFault;
+    public void reset() throws AxisFault {
+        if (!completed) {
+            throw new AxisFault(Messages.getMessage("cannotreset"));
+        }
+        oc = null;
+        completed = false;
+    }
+
 
     /**
      * To close the transport if necessary , can call this method. The main 
@@ -120,11 +172,90 @@
      * @param msgCtxt : MessageContext# which has all the transport information
      * @throws AxisFault : throws AxisFault if something goes wrong
      */
-    public void complete(MessageContext msgCtxt) throws AxisFault;
+    public void complete(MessageContext msgCtxt) throws AxisFault {
+        TransportOutDescription trsout = msgCtxt.getTransportOut();
+        if (trsout != null) {
+            trsout.getSender().cleanup(msgCtxt);
+        }
+    }
+
 
     /**
      * To get the operation context of the operation client
+     *
      * @return OperationContext
      */
-    public OperationContext getOperationContext();
+    public OperationContext getOperationContext() {
+        return oc;
+    }
+
+    /**
+     * Create a message ID for the given message context if needed. If user gives an option with
+     * MessageID then just copy that into MessageContext , and with that there can be multiple
+     * message with same MessageID unless user call setOption for each invocation.
+     * <p/>
+     * If user want to give message ID then the better way is to set the message ID in the option and
+     * call setOption for each invocation then the right thing will happen.
+     * <p/>
+     * If user does not give a message ID then the new one will be created and set that into Message
+     * Context.
+     *
+     * @param mc the message context whose id is to be set
+     */
+    protected void setMessageID(MessageContext mc) {
+        // now its the time to put the parameters set by the user in to the
+        // correct places and to the
+        // if there is no message id still, set a new one.
+        String messageId = options.getMessageId();
+        if (messageId == null || "".equals(messageId)) {
+            messageId = UUIDGenerator.getUUID();
+        }
+        mc.setMessageID(messageId);
+    }
+
+    protected void addReferenceParameters(MessageContext msgctx) {
+        EndpointReference to = msgctx.getTo();
+        if (options.isManageSession()) {
+            EndpointReference tepr = sc.getTargetEPR();
+            if (tepr != null) {
+                Map map = tepr.getAllReferenceParameters();
+                if (map != null) {
+                    Iterator valuse = map.values().iterator();
+                    while (valuse.hasNext()) {
+                        Object refparaelement = valuse.next();
+                        if (refparaelement instanceof OMElement) {
+                            to.addReferenceParameter((OMElement) refparaelement);
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    protected void prepareMessageContext(ConfigurationContext cc, MessageContext mc)  throws AxisFault{
+        // set options on the message context
+        if (mc.getSoapAction() == null || "".equals(mc.getSoapAction())) {
+            mc.setSoapAction(options.getAction());
+        }
+        mc.setOptions(options);
+        mc.setAxisMessage(axisOp.getMessage(WSDLConstants.MESSAGE_LABEL_OUT_VALUE));
+
+        // do Target Resolution
+        TargetResolver targetResolver = cc.getAxisConfiguration().getTargetResolverChain();
+        if (targetResolver != null) {
+            targetResolver.resolveTarget(mc);
+        }
+        // if the transport to use for sending is not specified, try to find it
+        // from the URL
+        TransportOutDescription senderTransport = options.getTransportOut();
+        if (senderTransport == null) {
+            EndpointReference toEPR = (options.getTo() != null) ? options
+                    .getTo() : mc.getTo();
+            senderTransport = ClientUtils.inferOutTransport(cc
+                    .getAxisConfiguration(), toEPR, mc);
+        }
+        mc.setTransportOut(senderTransport);
+
+        addReferenceParameters(mc);
+    }
 }

Modified: webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/InOutAxisOperation.java
URL: http://svn.apache.org/viewvc/webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/InOutAxisOperation.java?view=diff&rev=487578&r1=487577&r2=487578
==============================================================================
--- webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/InOutAxisOperation.java (original)
+++ webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/InOutAxisOperation.java Fri Dec 15 07:08:05 2006
@@ -16,41 +16,21 @@
 package org.apache.axis2.description;
 
 import org.apache.axis2.AxisFault;
-import org.apache.axis2.wsdl.WSDLConstants;
 import org.apache.axis2.context.MessageContext;
 import org.apache.axis2.context.OperationContext;
 import org.apache.axis2.i18n.Messages;
 
 import javax.xml.namespace.QName;
-import java.util.ArrayList;
 import java.util.HashMap;
 
-public class InOutAxisOperation extends AxisOperation {
-    private AxisMessage inFaultMessage;
-//    private AxisMessage inMessage;
-    private AxisMessage outFaultMessage;
-//    private AxisMessage outMessage;
+public class InOutAxisOperation extends TwoChannelAxisOperation {
 
     public InOutAxisOperation() {
         super();
-        createMessages();
     }
 
     public InOutAxisOperation(QName name) {
         super(name);
-        createMessages();
-    }
-
-    public void addMessage(AxisMessage message, String label) {
-        if (WSDLConstants.MESSAGE_LABEL_OUT_VALUE.equals(label)) {
-            addChild("outMessage", message);
-        } else if (WSDLConstants.MESSAGE_LABEL_IN_VALUE.equals(label)) {
-            addChild("inMessage", message);
-        } else if (WSDLConstants.MESSAGE_LABEL_FAULT_VALUE.equals(label)) {
-            addChild("faultMessage", message);
-        } else {
-            throw new UnsupportedOperationException("Not yet implemented");
-        }
     }
 
     public void addMessageContext(MessageContext msgContext, OperationContext opContext)
@@ -70,89 +50,5 @@
             opContext.setComplete(true);
             opContext.cleanup();
         }
-    }
-
-    public void addFaultMessageContext(MessageContext msgContext, OperationContext opContext) throws AxisFault {
-        HashMap mep = opContext.getMessageContexts();
-        MessageContext faultMessageCtxt = (MessageContext) mep.get(MESSAGE_LABEL_FAULT_VALUE);
-
-        if (faultMessageCtxt != null) {
-             throw new AxisFault(Messages.getMessage("mepcompleted"));
-        } else {
-            mep.put(MESSAGE_LABEL_FAULT_VALUE, msgContext);
-            opContext.setComplete(true);
-            opContext.cleanup();
-        }
-
-    }
-
-    private void createMessages() {
-//        inMessage = new AxisMessage();
-//        inMessage.setDirection(WSDLConstants.WSDL_MESSAGE_DIRECTION_IN);
-//        inMessage.setParent(this);
-
-        AxisMessage inMessage = new AxisMessage();
-        inMessage.setDirection(WSDLConstants.WSDL_MESSAGE_DIRECTION_IN);
-        inMessage.setParent(this);
-        addChild("inMessage", inMessage);
-
-        inFaultMessage = new AxisMessage();
-        inFaultMessage.setParent(this);
-
-        outFaultMessage = new AxisMessage();
-        outFaultMessage.setParent(this);
-
-//        outMessage = new AxisMessage();
-//        outMessage.setDirection(WSDLConstants.WSDL_MESSAGE_DIRECTION_OUT);
-//        outMessage.setParent(this);
-
-        AxisMessage outMessage = new AxisMessage();
-        outMessage.setDirection(WSDLConstants.WSDL_MESSAGE_DIRECTION_OUT);
-        outMessage.setParent(this);
-        addChild("outMessage", outMessage);
-
-
-    }
-
-    public AxisMessage getMessage(String label) {
-        if (WSDLConstants.MESSAGE_LABEL_OUT_VALUE.equals(label)) {
-            return (AxisMessage) getChild("outMessage");
-        } else if (WSDLConstants.MESSAGE_LABEL_IN_VALUE.equals(label)) {
-            return (AxisMessage) getChild("inMessage");
-        } else {
-            throw new UnsupportedOperationException("Not yet implemented");
-        }
-    }
-
-    public ArrayList getPhasesInFaultFlow() {
-        return inFaultMessage.getMessageFlow();
-    }
-
-    public ArrayList getPhasesOutFaultFlow() {
-        return outFaultMessage.getMessageFlow();
-    }
-
-    public ArrayList getPhasesOutFlow() {
-        return ((AxisMessage) getChild("outMessage")).getMessageFlow();
-    }
-
-    public ArrayList getRemainingPhasesInFlow() {
-        return ((AxisMessage) getChild("inMessage")).getMessageFlow();
-    }
-
-    public void setPhasesInFaultFlow(ArrayList list) {
-        inFaultMessage.setMessageFlow(list);
-    }
-
-    public void setPhasesOutFaultFlow(ArrayList list) {
-        outFaultMessage.setMessageFlow(list);
-    }
-
-    public void setPhasesOutFlow(ArrayList list) {
-        ((AxisMessage) getChild("outMessage")).setMessageFlow(list);
-    }
-
-    public void setRemainingPhasesInFlow(ArrayList list) {
-        ((AxisMessage) getChild("inMessage")).setMessageFlow(list);
     }
 }

Modified: webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/OutInAxisOperation.java
URL: http://svn.apache.org/viewvc/webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/OutInAxisOperation.java?view=diff&rev=487578&r1=487577&r2=487578
==============================================================================
--- webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/OutInAxisOperation.java (original)
+++ webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/OutInAxisOperation.java Fri Dec 15 07:08:05 2006
@@ -15,7 +15,6 @@
 */
 package org.apache.axis2.description;
 
-import org.apache.axiom.om.OMElement;
 import org.apache.axiom.soap.SOAPBody;
 import org.apache.axiom.soap.SOAPEnvelope;
 import org.apache.axiom.soap.SOAPFault;
@@ -34,16 +33,12 @@
 import org.apache.axis2.i18n.Messages;
 import org.apache.axis2.transport.TransportUtils;
 import org.apache.axis2.util.CallbackReceiver;
-import org.apache.axis2.util.UUIDGenerator;
-import org.apache.axis2.util.TargetResolver;
 import org.apache.axis2.wsdl.WSDLConstants;
 
 import javax.xml.namespace.QName;
 import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
 
-public class OutInAxisOperation extends InOutAxisOperation {
+public class OutInAxisOperation extends TwoChannelAxisOperation {
     public OutInAxisOperation() {
         super();
         setMessageExchangePattern(WSDL20_2004Constants.MEP_URI_OUT_IN);
@@ -94,53 +89,11 @@
 /**
  * MEP client for moi.
  */
-class OutInAxisOperationClient implements OperationClient {
-
-    private AxisOperation axisOp;
-
-    protected ServiceContext sc;
-
-    protected Options options;
-
-    protected OperationContext oc;
-
-    protected Callback callback;
-
-    /*
-     * indicates whether the MEP execution has completed (and hence ready for
-     * resetting)
-     */
-    boolean completed;
+class OutInAxisOperationClient extends OperationClient {
 
     OutInAxisOperationClient(OutInAxisOperation axisOp, ServiceContext sc,
                              Options options) {
-        this.axisOp = axisOp;
-        this.sc = sc;
-        this.options = options;
-        this.completed = false;
-        this.oc = sc.createOperationContext(axisOp);
-    }
-
-    /**
-     * Sets the options that should be used for this particular client. This
-     * resets the entire set of options to use the new options - so you'd lose
-     * any option cascading that may have been set up.
-     *
-     * @param options the options
-     */
-    public void setOptions(Options options) {
-        this.options = options;
-    }
-
-    /**
-     * Returns the options used by this client. If you want to set a single
-     * option, then the right way is to call getOptions() and set specific
-     * options.
-     *
-     * @return Returns the options, which will never be null.
-     */
-    public Options getOptions() {
-        return options;
+        super(axisOp, sc, options);
     }
 
     /**
@@ -177,30 +130,6 @@
         this.callback = callback;
     }
 
-    /**
-     * Create a message ID for the given message context if needed. If user gives an option with
-     * MessageID then just copy that into MessageContext , and with that there can be multiple
-     * message with same MessageID unless user call setOption for each invocation.
-     * <p/>
-     * If user want to give message ID then the better way is to set the message ID in the option and
-     * call setOption for each invocation then the right thing will happen.
-     * <p/>
-     * If user does not give a message ID then the new one will be created and set that into Message
-     * Context.
-     *
-     * @param mc the message context whose id is to be set
-     */
-    private void setMessageID(MessageContext mc) {
-        // now its the time to put the parameters set by the user in to the
-        // correct places and to the
-        // if there is no message id still, set a new one.
-        String messageId = options.getMessageId();
-        if (messageId == null || "".equals(messageId)) {
-            messageId = UUIDGenerator.getUUID();
-        }
-        mc.setMessageID(messageId);
-    }
-
 
     /**
      * Executes the MEP. What this does depends on the specific MEP client. The
@@ -227,30 +156,7 @@
         if (mc == null) {
             throw new AxisFault(Messages.getMessage("outmsgctxnull"));
         }
-        //setting AxisMessage
-        mc.setAxisMessage(axisOp.getMessage(WSDLConstants.MESSAGE_LABEL_OUT_VALUE));
-        if (mc.getSoapAction() == null || "".equals(mc.getSoapAction())) {
-            mc.setSoapAction(options.getAction());
-        }
-        mc.setOptions(options);
-
-        // do Target Resolution
-        TargetResolver targetResolver = cc.getAxisConfiguration().getTargetResolverChain();
-        if(targetResolver != null){
-            targetResolver.resolveTarget(mc);
-        }
-
-        // if the transport to use for sending is not specified, try to find it
-        // from the URL
-        TransportOutDescription transportOut = options.getTransportOut();
-        if (transportOut == null) {
-            EndpointReference toEPR = (options.getTo() != null) ? options
-                    .getTo() : mc.getTo();
-            transportOut = ClientUtils.inferOutTransport(cc
-                    .getAxisConfiguration(), toEPR, mc);
-        }
-        mc.setTransportOut(transportOut);
-
+        prepareMessageContext(cc, mc);
 
         if (options.getTransportIn() == null && mc.getTransportIn() == null) {
             mc.setTransportIn(ClientUtils.inferInTransport(cc
@@ -259,7 +165,6 @@
             mc.setTransportIn(options.getTransportIn());
         }
 
-        addReferenceParameters(mc);
         if (options.isUseSeparateListener()) {
             CallbackReceiver callbackReceiver = (CallbackReceiver) axisOp
                     .getMessageReceiver();
@@ -322,24 +227,6 @@
         }
     }
 
-    private void addReferenceParameters(MessageContext msgctx) {
-        EndpointReference to = msgctx.getTo();
-        if (options.isManageSession()) {
-            EndpointReference tepr = sc.getTargetEPR();
-            if (tepr != null) {
-                Map map = tepr.getAllReferenceParameters();
-                if (map != null) {
-                    Iterator valuse = map.values().iterator();
-                    while (valuse.hasNext()) {
-                        Object refparaelement = valuse.next();
-                        if (refparaelement instanceof OMElement) {
-                            to.addReferenceParameter((OMElement) refparaelement);
-                        }
-                    }
-                }
-            }
-        }
-    }
 
     /**
      * @param msgctx
@@ -393,33 +280,6 @@
             }
         }
         return responseMessageContext;
-    }
-
-    /**
-     * Resets the MEP client to a clean status after the MEP has completed. This
-     * is how you can reuse a MEP client. NOTE: this does not reset the options;
-     * only the internal state so the client can be used again.
-     *
-     * @throws AxisFault if reset is called before the MEP client has completed an
-     *                   interaction.
-     */
-    public void reset() throws AxisFault {
-        if (!completed) {
-            throw new AxisFault(Messages.getMessage("cannotreset"));
-        }
-        oc = null;
-        completed = false;
-    }
-
-    public void complete(MessageContext msgCtxt) throws AxisFault {
-        TransportOutDescription trsout = msgCtxt.getTransportOut();
-        if (trsout != null) {
-            trsout.getSender().cleanup(msgCtxt);
-        }
-    }
-
-    public OperationContext getOperationContext() {
-        return oc;
     }
 
     /**

Modified: webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/OutOnlyAxisOperation.java
URL: http://svn.apache.org/viewvc/webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/OutOnlyAxisOperation.java?view=diff&rev=487578&r1=487577&r2=487578
==============================================================================
--- webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/OutOnlyAxisOperation.java (original)
+++ webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/OutOnlyAxisOperation.java Fri Dec 15 07:08:05 2006
@@ -15,9 +15,7 @@
 */
 package org.apache.axis2.description;
 
-import org.apache.axiom.om.OMElement;
 import org.apache.axis2.AxisFault;
-import org.apache.axis2.addressing.EndpointReference;
 import org.apache.axis2.client.OperationClient;
 import org.apache.axis2.client.Options;
 import org.apache.axis2.client.async.Callback;
@@ -27,15 +25,11 @@
 import org.apache.axis2.context.ServiceContext;
 import org.apache.axis2.engine.AxisEngine;
 import org.apache.axis2.i18n.Messages;
-import org.apache.axis2.util.UUIDGenerator;
-import org.apache.axis2.util.TargetResolver;
 import org.apache.axis2.wsdl.WSDLConstants;
 
 import javax.xml.namespace.QName;
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
 
 public class OutOnlyAxisOperation extends AxisOperation {
 
@@ -162,52 +156,13 @@
 /**
  * MEP client for moi.
  */
-class OutOnlyAxisOperationClient implements OperationClient {
-    OutOnlyAxisOperation axisOp;
+class OutOnlyAxisOperationClient extends OperationClient {
 
-    ServiceContext sc;
-
-    Options options;
-
-    MessageContext mc;
-
-    OperationContext oc;
-
-    /*
-    * indicates whether the MEP execution has completed (and hence ready for
-    * resetting)
-    */
-    boolean completed;
+    private MessageContext mc;
 
     OutOnlyAxisOperationClient(OutOnlyAxisOperation axisOp, ServiceContext sc,
                                Options options) {
-        this.axisOp = axisOp;
-        this.sc = sc;
-        this.options = options;
-        this.completed = false;
-        oc = sc.createOperationContext(axisOp);
-    }
-
-    /**
-     * Sets the options that should be used for this particular client. This
-     * resets the entire set of options to use the new options - so you'd lose
-     * any option cascading that may have been set up.
-     *
-     * @param options the options
-     */
-    public void setOptions(Options options) {
-        this.options = options;
-    }
-
-    /**
-     * Returns the options used by this client. If you want to set a single
-     * option, then the right way is to do getOptions() and set specific
-     * options.
-     *
-     * @return Returns the options, which will never be null.
-     */
-    public Options getOptions() {
-        return options;
+        super(axisOp, sc, options);
     }
 
     /**
@@ -260,48 +215,6 @@
     }
 
     /**
-     * Create a message ID for the given message context if needed. If user gives an option with
-     * MessageID then just copy that into MessageContext , and with that there can be multiple
-     * message with same MessageID unless user call setOption for each invocation.
-     * <p/>
-     * If user want to give message ID then the better way is to set the message ID in the option and
-     * call setOption for each invocation then the right thing will happen.
-     * <p/>
-     * If user does not give a message ID then the new one will be created and set that into Message
-     * Context.
-     *
-     * @param mc the message context whose id is to be set
-     */
-    private void setMessageID(MessageContext mc) {
-        // now its the time to put the parameters set by the user in to the
-        // correct places and to the
-        // if there is no message id still, set a new one.
-        String messageId = options.getMessageId();
-        if (messageId == null || "".equals(messageId)) {
-            messageId = UUIDGenerator.getUUID();
-        }
-        mc.setMessageID(messageId);
-    }
-
-    private void addReferenceParameters(MessageContext msgctx) {
-        EndpointReference to = msgctx.getTo();
-        if (options.isManageSession()) {
-            EndpointReference tepr = sc.getTargetEPR();
-            if (tepr != null) {
-                Map map = tepr.getAllReferenceParameters();
-                Iterator valuse = map.values().iterator();
-                while (valuse.hasNext()) {
-                    Object refparaelement = valuse.next();
-                    if (refparaelement instanceof OMElement) {
-                        to.addReferenceParameter((OMElement) refparaelement);
-                    }
-                }
-            }
-        }
-    }
-
-
-    /**
      * Executes the MEP. What this does depends on the specific MEP client. The
      * basic idea is to have the MEP client execute and do something with the
      * messages that have been added to it so far. For example, if its an Out-In
@@ -319,36 +232,12 @@
             throw new AxisFault(Messages.getMessage("mepiscomplted"));
         }
         ConfigurationContext cc = sc.getConfigurationContext();
-
-        // set options on the message context
-        if (mc.getSoapAction() == null || "".equals(mc.getSoapAction())) {
-            mc.setSoapAction(options.getAction());
-        }
-        mc.setOptions(options);
-        mc.setAxisMessage(axisOp.getMessage(WSDLConstants.MESSAGE_LABEL_OUT_VALUE));
-
-        // do Target Resolution
-        TargetResolver targetResolver = cc.getAxisConfiguration().getTargetResolverChain();
-        if(targetResolver != null){
-            targetResolver.resolveTarget(mc);
-        }
-
+        prepareMessageContext(cc, mc);
         // setting message ID if it null
-        // if the transport to use for sending is not specified, try to find it
-        // from the URL
-        TransportOutDescription senderTransport = options.getTransportOut();
-        if (senderTransport == null) {
-            EndpointReference toEPR = (options.getTo() != null) ? options
-                    .getTo() : mc.getTo();
-            senderTransport = ClientUtils.inferOutTransport(cc
-                    .getAxisConfiguration(), toEPR, mc);
-        }
-        mc.setTransportOut(senderTransport);
 
         // create the operation context for myself
         OperationContext oc = new OperationContext(axisOp, sc);
         oc.addMessageContext(mc);
-        addReferenceParameters(mc);
         // ship it out
         AxisEngine engine = new AxisEngine(cc);
         if (!block) {
@@ -358,32 +247,4 @@
         // all done
         completed = true;
     }
-
-    /**
-     * Resets the MEP client to a clean status after the MEP has completed. This
-     * is how you can reuse a MEP client. NOTE: this does not reset the options;
-     * only the internal state so the client can be used again.
-     *
-     * @throws AxisFault if reset is called before the MEP client has completed an
-     *                   interaction.
-     */
-    public void reset() throws AxisFault {
-        if (!completed) {
-            throw new AxisFault(Messages.getMessage("cannotreset"));
-        }
-        mc = null;
-        completed = false;
-    }
-
-    public void complete(MessageContext msgCtxt) throws AxisFault {
-        TransportOutDescription trsout = msgCtxt.getTransportOut();
-        if (trsout != null) {
-            trsout.getSender().cleanup(msgCtxt);
-        }
-    }
-
-    public OperationContext getOperationContext() {
-        return oc;
-    }
-
 }

Added: webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/TwoChannelAxisOperation.java
URL: http://svn.apache.org/viewvc/webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/TwoChannelAxisOperation.java?view=auto&rev=487578
==============================================================================
--- webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/TwoChannelAxisOperation.java (added)
+++ webservices/axis2/trunk/java/modules/kernel/src/org/apache/axis2/description/TwoChannelAxisOperation.java Fri Dec 15 07:08:05 2006
@@ -0,0 +1,136 @@
+package org.apache.axis2.description;
+
+import org.apache.axis2.wsdl.WSDLConstants;
+import org.apache.axis2.context.MessageContext;
+import org.apache.axis2.context.OperationContext;
+import org.apache.axis2.AxisFault;
+import org.apache.axis2.i18n.Messages;
+
+import javax.xml.namespace.QName;
+import java.util.ArrayList;
+import java.util.HashMap;
+/*
+* Copyright 2004,2005 The Apache Software Foundation.
+*
+* Licensed under the Apache License, Version 2.0 (the "License");
+* you may not use this file except in compliance with the License.
+* You may obtain a copy of the License at
+*
+*      http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*
+*
+*/
+
+/**
+ * This class is to keep common methods and properties in InOut and OutIn axisOperation
+ */
+public abstract class TwoChannelAxisOperation extends AxisOperation {
+
+    protected AxisMessage inFaultMessage;
+    protected AxisMessage outFaultMessage;
+    protected AxisMessage inMessage;
+    protected AxisMessage outMessage;
+
+    public TwoChannelAxisOperation() {
+        super();
+        createMessages();
+    }
+
+    public TwoChannelAxisOperation(QName name) {
+        super(name);
+        createMessages();
+    }
+
+    private void createMessages() {
+        inMessage = new AxisMessage();
+        inMessage.setDirection(WSDLConstants.WSDL_MESSAGE_DIRECTION_IN);
+        inMessage.setParent(this);
+        addChild("inMessage", inMessage);
+
+        inFaultMessage = new AxisMessage();
+        inFaultMessage.setParent(this);
+
+        outFaultMessage = new AxisMessage();
+        outFaultMessage.setParent(this);
+
+        outMessage = new AxisMessage();
+        outMessage.setDirection(WSDLConstants.WSDL_MESSAGE_DIRECTION_OUT);
+        outMessage.setParent(this);
+        addChild("outMessage", outMessage);
+    }
+
+    public void addMessage(AxisMessage message, String label) {
+           if (WSDLConstants.MESSAGE_LABEL_OUT_VALUE.equals(label)) {
+               addChild("outMessage", message);
+           } else if (WSDLConstants.MESSAGE_LABEL_IN_VALUE.equals(label)) {
+               addChild("inMessage", message);
+           } else if (WSDLConstants.MESSAGE_LABEL_FAULT_VALUE.equals(label)) {
+               addChild("faultMessage", message);
+           } else {
+               throw new UnsupportedOperationException("Not yet implemented");
+           }
+       }
+
+
+    public AxisMessage getMessage(String label) {
+        if (WSDLConstants.MESSAGE_LABEL_OUT_VALUE.equals(label)) {
+            return (AxisMessage) getChild("outMessage");
+        } else if (WSDLConstants.MESSAGE_LABEL_IN_VALUE.equals(label)) {
+            return (AxisMessage) getChild("inMessage");
+        } else {
+            throw new UnsupportedOperationException("Not yet implemented");
+        }
+    }
+
+    public void addFaultMessageContext(MessageContext msgContext, OperationContext opContext) throws AxisFault {
+        HashMap mep = opContext.getMessageContexts();
+        MessageContext faultMessageCtxt = (MessageContext) mep.get(MESSAGE_LABEL_FAULT_VALUE);
+
+        if (faultMessageCtxt != null) {
+            throw new AxisFault(Messages.getMessage("mepcompleted"));
+        } else {
+            mep.put(MESSAGE_LABEL_FAULT_VALUE, msgContext);
+            opContext.setComplete(true);
+            opContext.cleanup();
+        }
+
+    }
+
+    public ArrayList getPhasesInFaultFlow() {
+        return inFaultMessage.getMessageFlow();
+    }
+
+    public ArrayList getPhasesOutFaultFlow() {
+        return outFaultMessage.getMessageFlow();
+    }
+
+    public ArrayList getPhasesOutFlow() {
+        return outMessage.getMessageFlow();
+    }
+
+    public ArrayList getRemainingPhasesInFlow() {
+        return inMessage.getMessageFlow();
+    }
+
+    public void setPhasesInFaultFlow(ArrayList list) {
+        inFaultMessage.setMessageFlow(list);
+    }
+
+    public void setPhasesOutFaultFlow(ArrayList list) {
+        outFaultMessage.setMessageFlow(list);
+    }
+
+    public void setPhasesOutFlow(ArrayList list) {
+        outMessage.setMessageFlow(list);
+    }
+
+    public void setRemainingPhasesInFlow(ArrayList list) {
+        inMessage.setMessageFlow(list);
+    }
+}



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