You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ode.apache.org by mr...@apache.org on 2008/08/22 00:43:12 UTC

svn commit: r687887 - in /ode/branches/APACHE_ODE_1.X: bpel-api/src/main/java/org/apache/ode/bpel/iapi/ bpel-runtime/src/main/java/org/apache/ode/bpel/engine/ bpel-schemas/src/main/xsd/ bpel-store/src/main/java/org/apache/ode/store/ bpel-test/src/test/...

Author: mriou
Date: Thu Aug 21 15:43:11 2008
New Revision: 687887

URL: http://svn.apache.org/viewvc?rev=687887&view=rev
Log:
ODE-364 Publish/Subscribe across processes

Added:
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/java/org/apache/ode/test/PubSubTest.java
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld.wsdl
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld1.bpel
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld2.bpel
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldExternal.wsdl
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldPub.bpel
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldPub.wsdl
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/deploy.xml
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/test.properties
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld.wsdl
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld1.bpel
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld2.bpel
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/deploy.xml
    ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/test.properties
Modified:
    ode/branches/APACHE_ODE_1.X/bpel-api/src/main/java/org/apache/ode/bpel/iapi/ProcessConf.java
    ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelEngineImpl.java
    ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelProcess.java
    ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java
    ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/MyRoleMessageExchangeImpl.java
    ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/PartnerLinkMyRoleImpl.java
    ode/branches/APACHE_ODE_1.X/bpel-schemas/src/main/xsd/dd.xsd
    ode/branches/APACHE_ODE_1.X/bpel-store/src/main/java/org/apache/ode/store/ProcessConfImpl.java

Modified: ode/branches/APACHE_ODE_1.X/bpel-api/src/main/java/org/apache/ode/bpel/iapi/ProcessConf.java
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-api/src/main/java/org/apache/ode/bpel/iapi/ProcessConf.java?rev=687887&r1=687886&r2=687887&view=diff
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-api/src/main/java/org/apache/ode/bpel/iapi/ProcessConf.java (original)
+++ ode/branches/APACHE_ODE_1.X/bpel-api/src/main/java/org/apache/ode/bpel/iapi/ProcessConf.java Thu Aug 21 15:43:11 2008
@@ -149,6 +149,13 @@
     Map<String, Endpoint> getInvokeEndpoints();
     
     /**
+     * Tells if the service is shareable
+     * @param serviceName
+     * @return true if the given service can be shared by processes
+     */
+    boolean isSharedService(QName serviceName);
+    
+    /**
      * Generic facility to get additional stuff out of the process descriptor. 
      * @param qname name of the extension element.
      * @return list of extension elements 

Modified: ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelEngineImpl.java
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelEngineImpl.java?rev=687887&r1=687886&r2=687887&view=diff
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelEngineImpl.java (original)
+++ ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelEngineImpl.java Thu Aug 21 15:43:11 2008
@@ -19,32 +19,47 @@
 
 package org.apache.ode.bpel.engine;
 
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.ObjectOutputStream;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.TimeUnit;
+
+import javax.wsdl.Operation;
+import javax.wsdl.PortType;
+import javax.xml.namespace.QName;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.dao.MessageExchangeDAO;
 import org.apache.ode.bpel.dao.ProcessDAO;
 import org.apache.ode.bpel.dao.ProcessInstanceDAO;
 import org.apache.ode.bpel.evt.BpelEvent;
+import org.apache.ode.bpel.iapi.BpelEngine;
+import org.apache.ode.bpel.iapi.BpelEngineException;
+import org.apache.ode.bpel.iapi.ContextException;
+import org.apache.ode.bpel.iapi.Endpoint;
+import org.apache.ode.bpel.iapi.Message;
+import org.apache.ode.bpel.iapi.MessageExchange;
+import org.apache.ode.bpel.iapi.MyRoleMessageExchange;
+import org.apache.ode.bpel.iapi.PartnerRoleMessageExchange;
+import org.apache.ode.bpel.iapi.Scheduler;
 import org.apache.ode.bpel.iapi.MessageExchange.MessageExchangePattern;
 import org.apache.ode.bpel.iapi.MessageExchange.Status;
 import org.apache.ode.bpel.iapi.MyRoleMessageExchange.CorrelationStatus;
-import org.apache.ode.bpel.iapi.*;
 import org.apache.ode.bpel.iapi.Scheduler.JobInfo;
 import org.apache.ode.bpel.intercept.MessageExchangeInterceptor;
 import org.apache.ode.bpel.o.OPartnerLink;
 import org.apache.ode.bpel.o.OProcess;
 import org.apache.ode.utils.msg.MessageBundle;
 
-import javax.wsdl.Operation;
-import javax.wsdl.PortType;
-import javax.xml.namespace.QName;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.ObjectOutputStream;
-import java.util.*;
-import java.util.concurrent.Callable;
-import java.util.concurrent.TimeUnit;
-
 /**
  * Implementation of the {@link BpelEngine} interface: provides the server methods that should be invoked in the context of a
  * transaction.
@@ -85,8 +100,11 @@
     final HashMap<QName, BpelProcess> _activeProcesses = new HashMap<QName, BpelProcess>();
 
     /** Mapping from myrole endpoint name to active process. */
-    private final HashMap<Endpoint, BpelProcess> _serviceMap = new HashMap<Endpoint, BpelProcess>();
+    private final HashMap<Endpoint, List<BpelProcess>> _serviceMap = new HashMap<Endpoint, List<BpelProcess>>();
 
+    /** Mapping from a potentially shared endpoint to its EPR */ 
+    private SharedEndpoints _sharedEps;     
+    
     /** Manage instance-level locks. */
     private final InstanceLockManager _instanceLockManager = new InstanceLockManager();
 
@@ -94,14 +112,40 @@
 
     public BpelEngineImpl(Contexts contexts) {
         _contexts = contexts;
+        _sharedEps = new SharedEndpoints();
+        _sharedEps.init();
     }
 
+    public SharedEndpoints getSharedEndpoints() {
+    	return _sharedEps;
+    }
+    
     public MyRoleMessageExchange createMessageExchange(String clientKey, QName targetService,
                                                        String operation, String pipedMexId)
             throws BpelEngineException {
 
-        BpelProcess target = route(targetService, null);
+        List<BpelProcess> targets = route(targetService, null);
 
+        if (targets == null || targets.size() == 0)
+            throw new BpelEngineException("NoSuchService: " + targetService);
+        
+        if (targets.size() == 1) {
+            // If the number of targets is one, create and return a simple MEX
+        	BpelProcess target = targets.get(0);
+        	return createNewMyRoleMex(target, clientKey, targetService, operation, pipedMexId);
+        } else {
+        	// If the number of targets is greater than one, create and return
+        	// a brokered MEX that embeds the simple MEXs for each of the targets
+        	BpelProcess template = targets.get(0);
+            ArrayList<MyRoleMessageExchange> meps = new ArrayList<MyRoleMessageExchange>();
+        	for (BpelProcess target : targets) {
+        		meps.add(createNewMyRoleMex(target, clientKey, targetService, operation, pipedMexId));
+        	}
+        	return createNewMyRoleMex(template, meps);	
+        }
+    }
+    
+    private MyRoleMessageExchange createNewMyRoleMex(BpelProcess target, String clientKey, QName targetService, String operation, String pipedMexId) {
         MessageExchangeDAO dao;
         if (target == null || target.isInMemory()) {
             dao = _contexts.inMemDao.getConnection().createMessageExchange(MessageExchangeDAO.DIR_PARTNER_INVOKES_MYROLE);
@@ -115,15 +159,32 @@
         dao.setStatus(Status.NEW.toString());
         dao.setOperation(operation);
         dao.setPipedMessageExchangeId(pipedMexId);
-        MyRoleMessageExchangeImpl mex = new MyRoleMessageExchangeImpl(this, dao);
+        MyRoleMessageExchangeImpl mex = new MyRoleMessageExchangeImpl(target, this, dao);
 
         if (target != null) {
             target.initMyRoleMex(mex);
         }
-
         return mex;
     }
 
+    /**
+     * Return a brokered MEX that delegates invocations to each of the embedded
+     * MEXs contained in the <code>meps</code> list, using the appropriate style.
+     * 
+     * @param target
+     * @param meps
+     * @param istyle
+     * @return
+     * @throws BpelEngineException
+     */
+    private MyRoleMessageExchange createNewMyRoleMex(BpelProcess target, List<MyRoleMessageExchange> meps) 
+    		throws BpelEngineException {
+    	MyRoleMessageExchangeImpl templateMex = (MyRoleMessageExchangeImpl) meps.get(0);
+    	MessageExchangeDAO templateMexDao = templateMex.getDAO();
+    	return new BrokeredMyRoleMessageExchangeImpl(target, this, meps, templateMexDao, templateMex);
+    }
+    
+    	
     public MyRoleMessageExchange createMessageExchange(String clientKey, QName targetService, String operation) {
         return createMessageExchange(clientKey, targetService, operation, null);        
     }
@@ -157,7 +218,7 @@
             }
             break;
         case MessageExchangeDAO.DIR_PARTNER_INVOKES_MYROLE:
-            mex = new MyRoleMessageExchangeImpl(this, mexdao);
+            mex = new MyRoleMessageExchangeImpl(process, this, mexdao);
             if (process != null) {
                 OPartnerLink plink = (OPartnerLink) process.getOProcess().getChild(mexdao.getPartnerLinkModelId());
                 PortType ptype = plink.myRolePortType;
@@ -181,14 +242,27 @@
                 __log.debug("Deactivating process " + p.getPID());
 
             Endpoint processEndpoint = null;
-            Iterator<Map.Entry<Endpoint,BpelProcess>> serviceIter = _serviceMap.entrySet().iterator();
-            while (serviceIter.hasNext()) {
-                Map.Entry<Endpoint,BpelProcess> processEntry = serviceIter.next();
-                if (processEntry.getValue()._pid.equals(process)) {
-                    serviceIter.remove();
-                    processEndpoint = processEntry.getKey();
-                }
-            }
+//            	for (Endpoint endpoint : _serviceMap.keySet()) {
+//            		List<BpelProcess> processes = _serviceMap.get(endpoint);
+//            		for (BpelProcess candidate : processes) {
+//            			if (candidate.getPID().equals(process)) {
+//            				processes.remove(candidate);
+//            				processEndpoint = endpoint;
+//            			}
+//            		}
+//            	}
+	            Iterator<Map.Entry<Endpoint,List<BpelProcess>>> serviceIter = _serviceMap.entrySet().iterator();
+	            while (serviceIter.hasNext()) {
+	                Map.Entry<Endpoint,List<BpelProcess>> processEntry = serviceIter.next();
+	                List<BpelProcess> entryProcesses = processEntry.getValue();
+	                for (int i = 0; i < entryProcesses.size(); i++) {
+	                	BpelProcess entryProcess = entryProcesses.get(i);
+	                    if (entryProcess.getPID().equals(process)) {
+	                        processEndpoint = processEntry.getKey();
+	                    	entryProcesses.remove(entryProcess);
+	                    }
+	                }
+	            }
 
             // Only deactivating if no other process (version) need that endpoint anymore
             // We're only routing using an endpoint/process map for now which means that deploying
@@ -198,12 +272,13 @@
             // A bit clunky, the maps held here should be retought a bit.
             boolean otherVersions = false;
             for (BpelProcess bpelProcess : _activeProcesses.values()) {
-                if (bpelProcess._pconf.getType().equals(p._pconf.getType()))
+                if (bpelProcess.getProcessType().equals(p.getProcessType()))
                     otherVersions = true;
             }
-            if (_serviceMap.get(processEndpoint) == null && !otherVersions) {
+            // Deactivate process anyway because it now checks for shared endpoints 
+//            if (_serviceMap.get(processEndpoint) == null && !otherVersions) {
                 p.deactivate();
-            }
+//            }
         }
         return p;
     }
@@ -220,7 +295,21 @@
         _activeProcesses.put(process.getPID(), process);
         for (Endpoint e : process.getServiceNames()) {
             __log.debug("Register process: serviceId=" + e + ", process=" + process);
-            _serviceMap.put(e, process);
+            List<BpelProcess> processes = _serviceMap.get(e);
+            if (processes == null) {
+            	processes = new ArrayList<BpelProcess>();
+            	_serviceMap.put(e, processes);
+            }
+            // Remove any older version of the process from the list
+            for (int i = 0; i < processes.size(); i++) {
+            	BpelProcess cachedVersion = processes.get(i);
+            	__log.debug("cached version " + cachedVersion.getPID() + " vs registering version " + process.getPID());
+            	if (cachedVersion.getProcessType().equals(process.getProcessType())) {
+            		processes.remove(cachedVersion);
+            		cachedVersion.deactivate();
+            	}
+            }
+            processes.add(process);
         }
         process.activate(this);
     }
@@ -235,11 +324,11 @@
      *            request message
      * @return process corresponding to the targetted service, or <code>null</code> if service identifier is not recognized.
      */
-    BpelProcess route(QName service, Message request) {
+    List<BpelProcess> route(QName service, Message request) {
         // TODO: use the message to route to the correct service if more than
         // one service is listening on the same endpoint.
 
-        BpelProcess routed = null;
+        List<BpelProcess> routed = null;
         for (Endpoint endpoint : _serviceMap.keySet()) {
             if (endpoint.serviceName.equals(service))
                 routed = _serviceMap.get(endpoint);

Modified: ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelProcess.java
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelProcess.java?rev=687887&r1=687886&r2=687887&view=diff
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelProcess.java (original)
+++ ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelProcess.java Thu Aug 21 15:43:11 2008
@@ -81,6 +81,9 @@
      * it's important to note that the same process with the same endpoint can have 2 different myroles. */
     private volatile Map<PartnerLinkMyRoleImpl, Endpoint> _endpointToMyRoleMap;
 
+    /** Mapping from a potentially shared endpoint to its EPR */ 
+    private SharedEndpoints _sharedEps;
+    
     // Backup hashmaps to keep initial endpoints handy after dehydration
     private Map<Endpoint, EndpointReference> _myEprs = new HashMap<Endpoint, EndpointReference>();
     private Map<Endpoint, EndpointReference> _partnerEprs = new HashMap<Endpoint, EndpointReference>();
@@ -497,14 +500,31 @@
 
     void activate(BpelEngineImpl engine) {
         _engine = engine;
+        _sharedEps = _engine.getSharedEndpoints();
         _debugger = new DebuggerSupport(this);
 
         __log.debug("Activating " + _pid);
         // Activate all the my-role endpoints.
         for (Map.Entry<String, Endpoint> entry : _pconf.getProvideEndpoints().entrySet()) {
-            EndpointReference initialEPR = _engine._contexts.bindingContext.activateMyRoleEndpoint(_pid, entry.getValue());
-            __log.debug("Activated " + _pid + " myrole " + entry.getKey() + ": EPR is " + initialEPR);
-            _myEprs.put(entry.getValue(), initialEPR);
+        	Endpoint endpoint = entry.getValue();
+        	EndpointReference initialEPR = null;
+        	if (isShareable(endpoint)) {
+	        	// Check if the EPR already exists for the given endpoint
+	        	initialEPR = _sharedEps.getEndpointReference(endpoint); 
+	        	if (initialEPR == null) {
+	        		// Create an EPR by physically activating the endpoint 
+	                initialEPR = _engine._contexts.bindingContext.activateMyRoleEndpoint(_pid, entry.getValue());
+	                _sharedEps.addEndpoint(endpoint, initialEPR);
+	                __log.debug("Activated " + _pid + " myrole " + entry.getKey() + ": EPR is " + initialEPR);
+	        	}
+	            // Increment the reference count on the endpoint  
+	            _sharedEps.incrementReferenceCount(endpoint);
+        	} else {
+        		// Create an EPR by physically activating the endpoint 
+                initialEPR = _engine._contexts.bindingContext.activateMyRoleEndpoint(_pid, entry.getValue());
+                __log.debug("Activated " + _pid + " myrole " + entry.getKey() + ": EPR is " + initialEPR);
+        	}
+            _myEprs.put(endpoint, initialEPR);
         }
         __log.debug("Activated " + _pid);
 
@@ -513,12 +533,43 @@
 
     void deactivate() {
         // Deactivate all the my-role endpoints.
-        for (Endpoint endpoint : _myEprs.keySet())
-            _engine._contexts.bindingContext.deactivateMyRoleEndpoint(endpoint);
-
+        for (Endpoint endpoint : _myEprs.keySet()) {
+        	// Deactivate the EPR only if there are no more references 
+        	// to this endpoint from any (active) BPEL process.
+        	if (isShareable(endpoint)) {
+	        	__log.debug("deactivating shared endpoint " + endpoint);
+	        	if (!_sharedEps.decrementReferenceCount(endpoint)) {
+		            _engine._contexts.bindingContext.deactivateMyRoleEndpoint(endpoint);
+		            _sharedEps.removeEndpoint(endpoint);
+	        	}
+        	} else {
+	        	__log.debug("deactivating non-shared endpoint " + endpoint);
+	            _engine._contexts.bindingContext.deactivateMyRoleEndpoint(endpoint);
+        	}
+        }
         // TODO Deactivate all the partner-role channels
     }
 
+    private boolean isShareable(Endpoint endpoint) {
+    	if (!_pconf.isSharedService(endpoint.serviceName)) {
+    		return false;
+    	}
+    	PartnerLinkMyRoleImpl partnerLink = null;
+    	if (_endpointToMyRoleMap == null) {
+    		return false;
+    	}
+    	for (Map.Entry<PartnerLinkMyRoleImpl, Endpoint> entry : _endpointToMyRoleMap.entrySet()) {
+    		if (entry.getValue().equals(endpoint)) {
+    			partnerLink = entry.getKey();
+    			break;
+    		}
+    	}
+    	if (partnerLink == null) {
+    		return false;    	
+    	}
+    	return partnerLink.isOneWayOnly();
+    }
+    
     EndpointReference getInitialPartnerRoleEPR(OPartnerLink link) {
         try {
             _hydrationLatch.latch(1);
@@ -659,6 +710,10 @@
     public long getLastUsed() {
         return _lastUsed;
     }
+    
+    QName getProcessType() {
+    	return _pconf.getType();
+    }    	 
 
     /**
      * Get a hint as to whether this process is hydrated. Note this is only a hint, since things could change.

Modified: ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java?rev=687887&r1=687886&r2=687887&view=diff
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java (original)
+++ ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java Thu Aug 21 15:43:11 2008
@@ -514,7 +514,7 @@
                 .getMessage().getQName());
         buildOutgoingMessage(message, msg);
 
-        MyRoleMessageExchangeImpl m = new MyRoleMessageExchangeImpl(_bpelProcess._engine, mex);
+        MyRoleMessageExchangeImpl m = new MyRoleMessageExchangeImpl(_bpelProcess, _bpelProcess._engine, mex);
         _bpelProcess.initMyRoleMex(m);
         m.setResponse(new MessageImpl(message));
 
@@ -730,12 +730,12 @@
                 partnerLink.partnerLink.partnerRolePortType, operation, partnerEpr, myRoleEndpoint, _bpelProcess
                 .getPartnerRoleChannel(partnerLink.partnerLink));
 
-        BpelProcess p2pProcess = null;
+        List<BpelProcess> p2pProcesses = null;
         Endpoint partnerEndpoint = _bpelProcess.getInitialPartnerRoleEndpoint(partnerLink.partnerLink);
         if (partnerEndpoint != null)
-            p2pProcess = _bpelProcess.getEngine().route(partnerEndpoint.serviceName, mex.getRequest());
+            p2pProcesses = _bpelProcess.getEngine().route(partnerEndpoint.serviceName, mex.getRequest());
 
-        if (p2pProcess != null) {
+        if (p2pProcesses != null && !p2pProcesses.isEmpty()) {
             // Creating a my mex using the same message id as partner mex to "pipe" them
             MyRoleMessageExchange myRoleMex = _bpelProcess.getEngine().createMessageExchange(
                     mex.getMessageExchangeId(), partnerEndpoint.serviceName,
@@ -1073,7 +1073,7 @@
         for (String mexId : mexRefs) {
             MessageExchangeDAO mexDao = _dao.getConnection().getMessageExchange(mexId);
             if (mexDao != null) {
-                MyRoleMessageExchangeImpl mex = new MyRoleMessageExchangeImpl(_bpelProcess._engine, mexDao);
+                MyRoleMessageExchangeImpl mex = new MyRoleMessageExchangeImpl(_bpelProcess, _bpelProcess._engine, mexDao);
                 switch (mex.getStatus()) {
                     case ASYNC:
                     case RESPONSE:
@@ -1098,7 +1098,7 @@
         for (String mexId : mexRefs) {
             MessageExchangeDAO mexDao = _dao.getConnection().getMessageExchange(mexId);
             if (mexDao != null) {
-                MyRoleMessageExchangeImpl mex = new MyRoleMessageExchangeImpl(_bpelProcess._engine, mexDao);
+                MyRoleMessageExchangeImpl mex = new MyRoleMessageExchangeImpl(_bpelProcess, _bpelProcess._engine, mexDao);
                 _bpelProcess.initMyRoleMex(mex);
 
                 Message message = mex.createMessage(faultData.getFaultName());
@@ -1118,7 +1118,7 @@
         for (String mexId : mexRefs) {
             MessageExchangeDAO mexDao = _dao.getConnection().getMessageExchange(mexId);
             if (mexDao != null) {
-                MyRoleMessageExchangeImpl mex = new MyRoleMessageExchangeImpl(_bpelProcess._engine, mexDao);
+                MyRoleMessageExchangeImpl mex = new MyRoleMessageExchangeImpl(_bpelProcess, _bpelProcess._engine, mexDao);
                 _bpelProcess.initMyRoleMex(mex);
                 mex.setFailure(FailureType.OTHER, "No response.", null);
                 _bpelProcess._engine._contexts.mexContext.onAsyncReply(mex);
@@ -1356,7 +1356,7 @@
                         + " on CKEY " + ckey);
             }
 
-            MyRoleMessageExchangeImpl mex = new MyRoleMessageExchangeImpl(_bpelProcess._engine, mexdao);
+            MyRoleMessageExchangeImpl mex = new MyRoleMessageExchangeImpl(_bpelProcess, _bpelProcess._engine, mexdao);
 
             inputMsgMatch(mroute.getGroupId(), mroute.getIndex(), mex);
             execute();

Modified: ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/MyRoleMessageExchangeImpl.java
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/MyRoleMessageExchangeImpl.java?rev=687887&r1=687886&r2=687887&view=diff
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/MyRoleMessageExchangeImpl.java (original)
+++ ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/MyRoleMessageExchangeImpl.java Thu Aug 21 15:43:11 2008
@@ -19,6 +19,16 @@
 
 package org.apache.ode.bpel.engine;
 
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import javax.xml.namespace.QName;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.dao.MessageExchangeDAO;
@@ -31,26 +41,22 @@
 import org.apache.ode.bpel.intercept.InterceptorInvoker;
 import org.apache.ode.bpel.intercept.MessageExchangeInterceptor;
 import org.apache.ode.bpel.intercept.MessageExchangeInterceptor.InterceptorContext;
-
-import javax.xml.namespace.QName;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
 
 class MyRoleMessageExchangeImpl extends MessageExchangeImpl implements MyRoleMessageExchange {
 
 
     private static final Log __log = LogFactory.getLog(MyRoleMessageExchangeImpl.class);
+    
+    protected BpelProcess _process;
 
     private static Map<String, ResponseCallback> _waitingCallbacks =
             new ConcurrentHashMap<String, ResponseCallback>();
 
-
-    public MyRoleMessageExchangeImpl(BpelEngineImpl engine, MessageExchangeDAO mexdao) {
+    public MyRoleMessageExchangeImpl(BpelProcess process, BpelEngineImpl engine, MessageExchangeDAO mexdao) {
         super(engine, mexdao);
+        _process = process;
     }
 
     public CorrelationStatus getCorrelationStatus() {
@@ -111,12 +117,12 @@
         if (!processInterceptors(this, InterceptorInvoker.__onBpelServerInvoked))
             return null;
 
-        BpelProcess target = _engine.route(getDAO().getCallee(), request);
+        List<BpelProcess> targets = _engine.route(getDAO().getCallee(), request);
 
         if (__log.isDebugEnabled())
-            __log.debug("invoke() EPR= " + _epr + " ==> " + target);
+            __log.debug("invoke() EPR= " + _epr + " ==> " + targets);
 
-        if (target == null) {
+        if (targets == null || targets.isEmpty()) {
             if (__log.isWarnEnabled())
                 __log.warn(__msgs.msgUnknownEPR("" + _epr));
 
@@ -124,24 +130,29 @@
             setFailure(MessageExchange.FailureType.UNKNOWN_ENDPOINT, null, null);
             return null;
         } else {
-            // Schedule a new job for invocation
-            WorkEvent we = new WorkEvent();
-            we.setType(WorkEvent.Type.INVOKE_INTERNAL);
-            if (target.isInMemory()) we.setInMem(true);
-            we.setProcessId(target.getPID());
-            we.setMexId(getDAO().getMessageExchangeId());
-
-            if (getOperation().getOutput() != null) {
-                ResponseCallback callback = new ResponseCallback();
-                _waitingCallbacks.put(getClientId(), callback);
-            }
-
-            setStatus(Status.ASYNC);
-            if (target.isInMemory())
-                _engine._contexts.scheduler.scheduleVolatileJob(true, we.getDetail());
-            else
-                _engine._contexts.scheduler.schedulePersistedJob(we.getDetail(), null);
-            return new ResponseFuture(getClientId());
+        	for (BpelProcess target : targets) {
+        		if (target.getPID().equals(_process.getPID())) {
+		            // Schedule a new job for invocation
+		            WorkEvent we = new WorkEvent();
+		            we.setType(WorkEvent.Type.INVOKE_INTERNAL);
+		            if (target.isInMemory()) we.setInMem(true);
+		            we.setProcessId(target.getPID());
+		            we.setMexId(getDAO().getMessageExchangeId());
+		
+		            if (getOperation().getOutput() != null) {
+		                ResponseCallback callback = new ResponseCallback();
+		                _waitingCallbacks.put(getClientId(), callback);
+		            }
+		
+		            setStatus(Status.ASYNC);
+		            if (target.isInMemory())
+		                _engine._contexts.scheduler.scheduleVolatileJob(true, we.getDetail());
+		            else
+		                _engine._contexts.scheduler.schedulePersistedJob(we.getDetail(), null);
+		            return new ResponseFuture(getClientId());
+        		}
+        	}
+        	return null;
         }
     }
 
@@ -172,6 +183,27 @@
     public boolean isAsynchronous() {
         return true;
     }
+    
+    /**
+     * Return a deep clone of the given message
+     * 
+     * @param message
+     * @return
+     */
+	protected Message cloneMessage(Message message) {
+		Message clone = createMessage(message.getType());
+		clone.setMessage((Element) message.getMessage().cloneNode(true));
+		Map<String, Node> headerParts = message.getHeaderParts();
+		for (String partName : headerParts.keySet()) {
+			clone.setHeaderPart(partName, (Element) headerParts.get(partName).cloneNode(true)); 
+		}
+		Map<String, Node> parts = message.getHeaderParts();
+		for (String partName : parts.keySet()) {
+			clone.setHeaderPart(partName, (Element) parts.get(partName).cloneNode(true)); 
+		}
+		return clone;
+	}
+    
 
     static class ResponseFuture implements Future {
         private String _clientId;

Modified: ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/PartnerLinkMyRoleImpl.java
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/PartnerLinkMyRoleImpl.java?rev=687887&r1=687886&r2=687887&view=diff
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/PartnerLinkMyRoleImpl.java (original)
+++ ode/branches/APACHE_ODE_1.X/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/PartnerLinkMyRoleImpl.java Thu Aug 21 15:43:11 2008
@@ -18,6 +18,16 @@
  */
 package org.apache.ode.bpel.engine;
 
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+
+import javax.wsdl.Operation;
+import javax.wsdl.PortType;
+import javax.xml.namespace.QName;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.common.CorrelationKey;
@@ -46,10 +56,6 @@
 import org.apache.ode.utils.msg.MessageBundle;
 import org.w3c.dom.Element;
 
-import javax.wsdl.Operation;
-import javax.xml.namespace.QName;
-import java.util.*;
-
 /**
  * @author Matthieu Riou <mriou at apache dot org>
  */
@@ -320,5 +326,17 @@
         CorrelationKey key = new CorrelationKey(cset.getId(), values);
         return key;
     }
-
+    
+    public boolean isOneWayOnly() {
+		PortType portType = _plinkDef.myRolePortType;
+		if (portType == null) {
+			return false;
+		}
+    	for (Operation operation : (List<Operation>) portType.getOperations()) {
+	    	if (operation.getOutput() != null) {
+	    		return false;
+	    	}
+    	}
+    	return true;
+    }    
 }

Modified: ode/branches/APACHE_ODE_1.X/bpel-schemas/src/main/xsd/dd.xsd
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-schemas/src/main/xsd/dd.xsd?rev=687887&r1=687886&r2=687887&view=diff
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-schemas/src/main/xsd/dd.xsd (original)
+++ ode/branches/APACHE_ODE_1.X/bpel-schemas/src/main/xsd/dd.xsd Thu Aug 21 15:43:11 2008
@@ -113,6 +113,7 @@
     <xs:complexType name="tProvide">
         <xs:sequence>
             <xs:element name="service" minOccurs="1" maxOccurs="1" type="dd:tService"/>
+			<xs:element name="enableSharing" minOccurs="0" maxOccurs="1" type="dd:tEnableSharing"/>
         </xs:sequence>
         <xs:attribute name="partnerLink" type="xs:string" use="required"/>
     </xs:complexType>
@@ -137,6 +138,13 @@
         <xs:attribute name="port" type="xs:NCName" use="required"/>
     </xs:complexType>
 
+    <xs:complexType name="tEnableSharing">
+        <xs:complexContent>
+			<xs:restriction base="xs:anyType">
+			</xs:restriction>
+		</xs:complexContent>
+    </xs:complexType>
+
     <xs:complexType name="tMexInterceptor" >
         <xs:annotation>
             <xs:documentation>

Modified: ode/branches/APACHE_ODE_1.X/bpel-store/src/main/java/org/apache/ode/store/ProcessConfImpl.java
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-store/src/main/java/org/apache/ode/store/ProcessConfImpl.java?rev=687887&r1=687886&r2=687887&view=diff
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-store/src/main/java/org/apache/ode/store/ProcessConfImpl.java (original)
+++ ode/branches/APACHE_ODE_1.X/bpel-store/src/main/java/org/apache/ode/store/ProcessConfImpl.java Thu Aug 21 15:43:11 2008
@@ -78,6 +78,7 @@
     private final HashMap<String, Endpoint> _partnerRoleInitialValues = new HashMap<String, Endpoint>();
 
     private final HashMap<String, Endpoint> _myRoleEndpoints = new HashMap<String, Endpoint>();
+    private final ArrayList<QName> _sharedServices = new ArrayList<QName>();
     private final Map<String, Set<BpelEvent.TYPE>> _events = new HashMap<String, Set<BpelEvent.TYPE>>();
     private final ArrayList<String> _mexi = new ArrayList<String>();
     ProcessState _state;
@@ -152,6 +153,10 @@
                 __log.debug("Processing <provide> element for process " + _pinfo.getName() + ": partnerlink " + plinkName + " --> "
                         + service.getName() + " : " + service.getPort());
                 _myRoleEndpoints.put(plinkName, new Endpoint(service.getName(), service.getPort()));
+
+                if (provide.isSetEnableSharing()) {
+                	_sharedServices.add(service.getName());
+                }
             }
         }
     }
@@ -248,6 +253,10 @@
         return Collections.unmodifiableMap(_myRoleEndpoints);
     }
 
+    public boolean isSharedService(QName serviceName) {
+    	return _sharedServices.contains(serviceName);
+    }
+    
     private void handleEndpoints() {
         // for (TProvide provide : _pinfo.getProvideList()) {
         // OPartnerLink pLink = _oprocess.getPartnerLink(provide.getPartnerLink());

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/java/org/apache/ode/test/PubSubTest.java
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/java/org/apache/ode/test/PubSubTest.java?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/java/org/apache/ode/test/PubSubTest.java (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/java/org/apache/ode/test/PubSubTest.java Thu Aug 21 15:43:11 2008
@@ -0,0 +1,32 @@
+/*
+ * 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 org.apache.ode.test;
+
+import org.junit.Test;
+
+public class PubSubTest extends BPELTestAbstract {
+	
+	@Test public void testPubSubInProc() throws Throwable {
+        go("/bpel/2.0/TestPubSubInProc");
+    }
+
+	@Test public void testPubSubOutOfProc() throws Throwable {
+        go("/bpel/2.0/TestPubSubOutOfProc");
+    }
+}

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld.wsdl
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld.wsdl?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld.wsdl (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld.wsdl Thu Aug 21 15:43:11 2008
@@ -0,0 +1,68 @@
+<?xml version="1.0" encoding="utf-8" ?>
+<!--
+  ~ 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.
+  -->
+
+<wsdl:definitions 
+    targetNamespace="http://ode/bpel/unit-test.wsdl"
+    xmlns="http://schemas.xmlsoap.org/wsdl/"
+    xmlns:tns="http://ode/bpel/unit-test.wsdl"
+    xmlns:wsdl="http://schemas.xmlsoap.org/wsdl/" 
+    xmlns:xsd="http://www.w3.org/2001/XMLSchema"
+    xmlns:soap="http://schemas.xmlsoap.org/wsdl/soap/"
+    xmlns:plnk="http://docs.oasis-open.org/wsbpel/2.0/plnktype">
+    
+    <wsdl:message name="HelloMessage">
+        <wsdl:part name="TestPart" type="xsd:string"/>
+    </wsdl:message>
+    
+    <wsdl:portType name="HelloPortType">
+        <wsdl:operation name="hello">
+            <wsdl:input message="tns:HelloMessage" name="TestIn"/>
+            <wsdl:output message="tns:HelloMessage" name="TestOut"/>
+        </wsdl:operation>    
+    </wsdl:portType>
+    
+     <wsdl:binding name="HelloSoapBinding" type="tns:HelloPortType">
+        <soap:binding style="rpc" transport="http://schemas.xmlsoap.org/soap/http"/>
+        <wsdl:operation name="hello">
+            <soap:operation soapAction="" style="rpc"/>
+            <wsdl:input>
+                <soap:body
+                    namespace="http://ode/bpel/unit-test.wsdl"
+                    use="literal"/>
+            </wsdl:input>
+            <wsdl:output>
+                <soap:body
+                    namespace="http://ode/bpel/unit-test.wsdl" 
+                    use="literal"/>
+            </wsdl:output>
+        </wsdl:operation>
+    </wsdl:binding>
+    <wsdl:service name="HelloService">
+		<wsdl:port name="HelloPort" binding="tns:HelloSoapBinding">
+     		<soap:address location="http://localhost:8080/ode/processes/helloWorld"/>			
+		</wsdl:port>
+    </wsdl:service>
+    
+   <plnk:partnerLinkType name="HelloPartnerLinkType">
+       <plnk:role name="me" portType="tns:HelloPortType"/>
+       <plnk:role name="you" portType="tns:HelloPortType"/>
+   </plnk:partnerLinkType>
+</wsdl:definitions>
+

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld1.bpel
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld1.bpel?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld1.bpel (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld1.bpel Thu Aug 21 15:43:11 2008
@@ -0,0 +1,68 @@
+<!--
+  ~ 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.
+  -->
+<process name="HelloWorld1"
+    targetNamespace="http://ode/bpel/unit-test" 
+    xmlns="http://docs.oasis-open.org/wsbpel/2.0/process/executable"
+    xmlns:tns="http://ode/bpel/unit-test"
+    xmlns:xsd="http://www.w3.org/2001/XMLSchema"
+    xmlns:test="http://ode/bpel/unit-test.wsdl"
+    queryLanguage="urn:oasis:names:tc:wsbpel:2.0:sublang:xpath2.0"
+    expressionLanguage="urn:oasis:names:tc:wsbpel:2.0:sublang:xpath2.0">
+
+  <import location="HelloWorld.wsdl"
+     namespace="http://ode/bpel/unit-test.wsdl"
+     importType="http://schemas.xmlsoap.org/wsdl/" />
+
+   <partnerLinks>
+      <partnerLink name="helloPartnerLink" 
+         partnerLinkType="test:HelloPartnerLinkType" 
+         myRole="me" />
+   </partnerLinks>
+    
+   <variables>
+     <variable name="myVar" messageType="test:HelloMessage"/>
+     <variable name="tmpVar" type="xsd:string"/>
+   </variables>
+
+   <sequence>
+       <receive
+          name="start"
+          partnerLink="helloPartnerLink"
+          portType="test:HelloPortType"
+          operation="hello"
+          variable="myVar"
+          createInstance="yes"/>
+
+      <assign name="assign1">
+          <copy>
+              <from variable="myVar" part="TestPart"/>
+              <to variable="tmpVar"/>
+          </copy>
+          <copy>
+              <from>concat($tmpVar,' World')</from>
+              <to variable="myVar" part="TestPart"/>
+          </copy>
+      </assign>
+       <reply name="end"  
+              partnerLink="helloPartnerLink"
+              portType="test:HelloPortType" 
+              operation="hello"
+              variable="myVar"/>
+   </sequence>
+</process>

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld2.bpel
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld2.bpel?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld2.bpel (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorld2.bpel Thu Aug 21 15:43:11 2008
@@ -0,0 +1,68 @@
+<!--
+  ~ 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.
+  -->
+<process name="HelloWorld2"
+    targetNamespace="http://ode/bpel/unit-test" 
+    xmlns="http://docs.oasis-open.org/wsbpel/2.0/process/executable"
+    xmlns:tns="http://ode/bpel/unit-test"
+    xmlns:xsd="http://www.w3.org/2001/XMLSchema"
+    xmlns:test="http://ode/bpel/unit-test.wsdl"
+    queryLanguage="urn:oasis:names:tc:wsbpel:2.0:sublang:xpath2.0"
+    expressionLanguage="urn:oasis:names:tc:wsbpel:2.0:sublang:xpath2.0">
+
+  <import location="HelloWorld.wsdl"
+     namespace="http://ode/bpel/unit-test.wsdl"
+     importType="http://schemas.xmlsoap.org/wsdl/" />
+
+   <partnerLinks>
+      <partnerLink name="helloPartnerLink" 
+         partnerLinkType="test:HelloPartnerLinkType" 
+         myRole="me" />
+   </partnerLinks>
+    
+   <variables>
+     <variable name="myVar" messageType="test:HelloMessage"/>
+     <variable name="tmpVar" type="xsd:string"/>
+   </variables>
+
+   <sequence>
+       <receive
+          name="start"
+          partnerLink="helloPartnerLink"
+          portType="test:HelloPortType"
+          operation="hello"
+          variable="myVar"
+          createInstance="yes"/>
+
+      <assign name="assign1">
+          <copy>
+              <from variable="myVar" part="TestPart"/>
+              <to variable="tmpVar"/>
+          </copy>
+          <copy>
+              <from>concat($tmpVar,' World')</from>
+              <to variable="myVar" part="TestPart"/>
+          </copy>
+      </assign>
+       <reply name="end"  
+              partnerLink="helloPartnerLink"
+              portType="test:HelloPortType" 
+              operation="hello"
+              variable="myVar"/>
+   </sequence>
+</process>

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldExternal.wsdl
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldExternal.wsdl?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldExternal.wsdl (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldExternal.wsdl Thu Aug 21 15:43:11 2008
@@ -0,0 +1,68 @@
+<?xml version="1.0" encoding="utf-8" ?>
+<!--
+  ~ 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.
+  -->
+
+<wsdl:definitions 
+    targetNamespace="http://ode/bpel/unit-test.wsdl"
+    xmlns="http://schemas.xmlsoap.org/wsdl/"
+    xmlns:tns="http://ode/bpel/unit-test.wsdl"
+    xmlns:wsdl="http://schemas.xmlsoap.org/wsdl/" 
+    xmlns:xsd="http://www.w3.org/2001/XMLSchema"
+    xmlns:soap="http://schemas.xmlsoap.org/wsdl/soap/"
+    xmlns:plnk="http://docs.oasis-open.org/wsbpel/2.0/plnktype">
+    
+    <wsdl:message name="HelloMessage">
+        <wsdl:part name="TestPart" type="xsd:string"/>
+    </wsdl:message>
+    
+    <wsdl:portType name="HelloPortType">
+        <wsdl:operation name="hello">
+            <wsdl:input message="tns:HelloMessage" name="TestIn"/>
+            <wsdl:output message="tns:HelloMessage" name="TestOut"/>
+        </wsdl:operation>    
+    </wsdl:portType>
+    
+     <wsdl:binding name="HelloSoapBinding" type="tns:HelloPortType">
+        <soap:binding style="rpc" transport="http://schemas.xmlsoap.org/soap/http"/>
+        <wsdl:operation name="hello">
+            <soap:operation soapAction="" style="rpc"/>
+            <wsdl:input>
+                <soap:body
+                    namespace="http://ode/bpel/unit-test.wsdl"
+                    use="literal"/>
+            </wsdl:input>
+            <wsdl:output>
+                <soap:body
+                    namespace="http://ode/bpel/unit-test.wsdl" 
+                    use="literal"/>
+            </wsdl:output>
+        </wsdl:operation>
+    </wsdl:binding>
+    <wsdl:service name="HelloPubService">
+		<wsdl:port name="HelloPubPort" binding="tns:HelloSoapBinding">
+     		<soap:address location="http://localhost:8080/ode/processes/HelloPubWorld"/>
+		</wsdl:port>
+    </wsdl:service>
+    
+   <plnk:partnerLinkType name="HelloPartnerLinkType">
+       <plnk:role name="me" portType="tns:HelloPortType"/>
+       <plnk:role name="you" portType="tns:HelloPortType"/>
+   </plnk:partnerLinkType>
+</wsdl:definitions>
+

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldPub.bpel
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldPub.bpel?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldPub.bpel (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldPub.bpel Thu Aug 21 15:43:11 2008
@@ -0,0 +1,18 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<bpel:process xmlns:bpel="http://docs.oasis-open.org/wsbpel/2.0/process/executable" xmlns:wsdl="http://schemas.xmlsoap.org/wsdl/" xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:vprop="http://docs.oasis-open.org/wsbpel/2.0/varprop" xmlns:pnlk="http://docs.oasis-open.org/wsbpel/2.0/plnktype" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xmlns:Start="http://ode/bpel/unit-test/Start" xmlns:HelloWorld="http://ode/bpel/unit-test/HelloWorld" xmlns:this="http://ode/bpel/unit-test" xmlns:diag="http://ode/bpel/unit-test.wsdl" xmlns:tns="http://ode/bpel/unit-test.wsdl" xmlns:xml="http://www.w3.org/XML/1998/namespace" xmlns:atomic="http://ode.apache.org/atomicScope" queryLanguage="urn:oasis:names:tc:wsbpel:2.0:sublang:xpath2.0" expressionLanguage="urn:oasis:names:tc:wsbpel:2.0:sublang:xpath2.0" name="HelloWorldPub" targetNamespace="http://ode/bpel/unit-test">
+  <bpel:import namespace="http://ode/bpel/unit-test.wsdl" location="HelloWorld.wsdl" importType="http://schemas.xmlsoap.org/wsdl/"/>
+  <bpel:import namespace="http://ode/bpel/unit-test.wsdl" location="HelloWorldPub.wsdl" importType="http://schemas.xmlsoap.org/wsdl/"/>
+  <bpel:partnerLinks>
+    <bpel:partnerLink name="helloWorldAndPoolForHelloPortPlkVar" partnerLinkType="diag:HelloWorldAndPoolForHelloPortPlk" initializePartnerRole="true" partnerRole="HelloWorld_for_Pool"/>
+    <bpel:partnerLink name="poolAndStartForHelloPortPlkVar" partnerLinkType="diag:PoolAndStartForHelloPortPlk" myRole="Pool_for_Start"/>
+  </bpel:partnerLinks>
+  <bpel:variables>
+    <bpel:variable name="tnsHelloRequestMsg" messageType="tns:HelloMessage"/>
+    <bpel:variable name="tnsHelloResponseMsg" messageType="tns:HelloMessage"/>
+  </bpel:variables>
+  <bpel:sequence>
+    <bpel:receive partnerLink="poolAndStartForHelloPortPlkVar" portType="tns:HelloPortType" operation="hello" variable="tnsHelloRequestMsg" createInstance="yes"/>
+	<bpel:invoke partnerLink="helloWorldAndPoolForHelloPortPlkVar" portType="tns:HelloPortType" operation="hello" inputVariable="tnsHelloRequestMsg" outputVariable="tnsHelloResponseMsg"/>
+    <bpel:reply partnerLink="poolAndStartForHelloPortPlkVar" portType="tns:HelloPortType" operation="hello" variable="tnsHelloResponseMsg"/>
+  </bpel:sequence>
+</bpel:process>
\ No newline at end of file

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldPub.wsdl
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldPub.wsdl?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldPub.wsdl (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/HelloWorldPub.wsdl Thu Aug 21 15:43:11 2008
@@ -0,0 +1,11 @@
+<?xml version='1.0' encoding='utf-8'?>
+<wsdl:definitions xmlns:bpdm="http://www.intalio/designer/business-process-data-modeling" xmlns:bpel="http://docs.oasis-open.org/wsbpel/2.0/process/executable" xmlns:diag="http://ode/bpel/unit-test.wsdl" xmlns:Start="http://ode/bpel/unit-test.wsdl/Start" xmlns:wsdl="http://schemas.xmlsoap.org/wsdl/" xmlns:tns="http://ode/bpel/unit-test.wsdl" xmlns:Pool="http://example.com/diagram/Pool" xmlns:xs="http://www.w3.org/2001/XMLSchema" xmlns:vprop="http://docs.oasis-open.org/wsbpel/2.0/varprop" xmlns:soap="http://schemas.xmlsoap.org/wsdl/soap/" xmlns:pnlk="http://docs.oasis-open.org/wsbpel/2.0/plnktype" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" targetNamespace="http://ode/bpel/unit-test.wsdl">
+    <wsdl:import namespace="http://ode/bpel/unit-test.wsdl" location="HelloWorld.wsdl"/>
+    <wsdl:import namespace="http://ode/bpel/unit-test.wsdl" location="HelloWorldExternal.wsdl"/>
+    <pnlk:partnerLinkType name="HelloWorldAndPoolForHelloPortPlk">
+        <pnlk:role name="HelloWorld_for_Pool" portType="tns:HelloPortType"/>
+    </pnlk:partnerLinkType>
+    <pnlk:partnerLinkType name="PoolAndStartForHelloPortPlk">
+        <pnlk:role name="Pool_for_Start" portType="tns:HelloPortType"/>
+    </pnlk:partnerLinkType>
+</wsdl:definitions>
\ No newline at end of file

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/deploy.xml
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/deploy.xml?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/deploy.xml (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/deploy.xml Thu Aug 21 15:43:11 2008
@@ -0,0 +1,49 @@
+<!--
+  ~ 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.
+  -->
+<deploy xmlns="http://www.apache.org/ode/schemas/dd/2007/03"
+	xmlns:pns="http://ode/bpel/unit-test" 
+	xmlns:wns="http://ode/bpel/unit-test.wsdl">
+
+	<process name="pns:HelloWorld1">
+		<active>true</active>
+		<provide partnerLink="helloPartnerLink">
+			<service name="wns:HelloService" port="HelloPort"/>
+			<enableSharing/>
+		</provide>
+	</process>
+
+	<process name="pns:HelloWorld2">
+		<active>true</active>
+		<provide partnerLink="helloPartnerLink">
+			<service name="wns:HelloService" port="HelloPort"/>
+			<enableSharing/>
+		</provide>
+	</process>
+	
+	<process name="pns:HelloWorldPub">
+		<active>true</active>
+		<provide partnerLink="poolAndStartForHelloPortPlkVar">
+			<service name="wns:HelloPubService" port="HelloPubPort"/>
+		</provide>
+		<invoke partnerLink="helloWorldAndPoolForHelloPortPlkVar">
+		    <service name="wns:HelloService" port="HelloPort"/>
+	    </invoke>
+  </process>
+	
+</deploy>

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/test.properties
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/test.properties?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/test.properties (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubInProc/test.properties Thu Aug 21 15:43:11 2008
@@ -0,0 +1,23 @@
+#
+#    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.
+#
+
+namespace=http://ode/bpel/unit-test.wsdl
+service=HelloPubService
+operation=hello
+request1=<message><TestPart><content>Hello</content></TestPart></message>
+response1=.*Hello World.*
+

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld.wsdl
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld.wsdl?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld.wsdl (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld.wsdl Thu Aug 21 15:43:11 2008
@@ -0,0 +1,68 @@
+<?xml version="1.0" encoding="utf-8" ?>
+<!--
+  ~ 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.
+  -->
+
+<wsdl:definitions 
+    targetNamespace="http://ode/bpel/unit-test.wsdl"
+    xmlns="http://schemas.xmlsoap.org/wsdl/"
+    xmlns:tns="http://ode/bpel/unit-test.wsdl"
+    xmlns:wsdl="http://schemas.xmlsoap.org/wsdl/" 
+    xmlns:xsd="http://www.w3.org/2001/XMLSchema"
+    xmlns:soap="http://schemas.xmlsoap.org/wsdl/soap/"
+    xmlns:plnk="http://docs.oasis-open.org/wsbpel/2.0/plnktype">
+    
+    <wsdl:message name="HelloMessage">
+        <wsdl:part name="TestPart" type="xsd:string"/>
+    </wsdl:message>
+    
+    <wsdl:portType name="HelloPortType">
+        <wsdl:operation name="hello">
+            <wsdl:input message="tns:HelloMessage" name="TestIn"/>
+            <wsdl:output message="tns:HelloMessage" name="TestOut"/>
+        </wsdl:operation>    
+    </wsdl:portType>
+    
+     <wsdl:binding name="HelloSoapBinding" type="tns:HelloPortType">
+        <soap:binding style="rpc" transport="http://schemas.xmlsoap.org/soap/http"/>
+        <wsdl:operation name="hello">
+            <soap:operation soapAction="" style="rpc"/>
+            <wsdl:input>
+                <soap:body
+                    namespace="http://ode/bpel/unit-test.wsdl"
+                    use="literal"/>
+            </wsdl:input>
+            <wsdl:output>
+                <soap:body
+                    namespace="http://ode/bpel/unit-test.wsdl" 
+                    use="literal"/>
+            </wsdl:output>
+        </wsdl:operation>
+    </wsdl:binding>
+    <wsdl:service name="HelloService">
+		<wsdl:port name="HelloPort" binding="tns:HelloSoapBinding">
+     		<soap:address location="http://localhost:8080/ode/processes/helloWorld"/>			
+		</wsdl:port>
+    </wsdl:service>
+    
+   <plnk:partnerLinkType name="HelloPartnerLinkType">
+       <plnk:role name="me" portType="tns:HelloPortType"/>
+       <plnk:role name="you" portType="tns:HelloPortType"/>
+   </plnk:partnerLinkType>
+</wsdl:definitions>
+

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld1.bpel
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld1.bpel?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld1.bpel (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld1.bpel Thu Aug 21 15:43:11 2008
@@ -0,0 +1,68 @@
+<!--
+  ~ 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.
+  -->
+<process name="HelloWorld1"
+    targetNamespace="http://ode/bpel/unit-test" 
+    xmlns="http://docs.oasis-open.org/wsbpel/2.0/process/executable"
+    xmlns:tns="http://ode/bpel/unit-test"
+    xmlns:xsd="http://www.w3.org/2001/XMLSchema"
+    xmlns:test="http://ode/bpel/unit-test.wsdl"
+    queryLanguage="urn:oasis:names:tc:wsbpel:2.0:sublang:xpath2.0"
+    expressionLanguage="urn:oasis:names:tc:wsbpel:2.0:sublang:xpath2.0">
+
+  <import location="HelloWorld.wsdl"
+     namespace="http://ode/bpel/unit-test.wsdl"
+     importType="http://schemas.xmlsoap.org/wsdl/" />
+
+   <partnerLinks>
+      <partnerLink name="helloPartnerLink" 
+         partnerLinkType="test:HelloPartnerLinkType" 
+         myRole="me" />
+   </partnerLinks>
+    
+   <variables>
+     <variable name="myVar" messageType="test:HelloMessage"/>
+     <variable name="tmpVar" type="xsd:string"/>
+   </variables>
+
+   <sequence>
+       <receive
+          name="start"
+          partnerLink="helloPartnerLink"
+          portType="test:HelloPortType"
+          operation="hello"
+          variable="myVar"
+          createInstance="yes"/>
+
+      <assign name="assign1">
+          <copy>
+              <from variable="myVar" part="TestPart"/>
+              <to variable="tmpVar"/>
+          </copy>
+          <copy>
+              <from>concat($tmpVar,' World')</from>
+              <to variable="myVar" part="TestPart"/>
+          </copy>
+      </assign>
+       <reply name="end"  
+              partnerLink="helloPartnerLink"
+              portType="test:HelloPortType" 
+              operation="hello"
+              variable="myVar"/>
+   </sequence>
+</process>

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld2.bpel
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld2.bpel?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld2.bpel (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/HelloWorld2.bpel Thu Aug 21 15:43:11 2008
@@ -0,0 +1,68 @@
+<!--
+  ~ 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.
+  -->
+<process name="HelloWorld2"
+    targetNamespace="http://ode/bpel/unit-test" 
+    xmlns="http://docs.oasis-open.org/wsbpel/2.0/process/executable"
+    xmlns:tns="http://ode/bpel/unit-test"
+    xmlns:xsd="http://www.w3.org/2001/XMLSchema"
+    xmlns:test="http://ode/bpel/unit-test.wsdl"
+    queryLanguage="urn:oasis:names:tc:wsbpel:2.0:sublang:xpath2.0"
+    expressionLanguage="urn:oasis:names:tc:wsbpel:2.0:sublang:xpath2.0">
+
+  <import location="HelloWorld.wsdl"
+     namespace="http://ode/bpel/unit-test.wsdl"
+     importType="http://schemas.xmlsoap.org/wsdl/" />
+
+   <partnerLinks>
+      <partnerLink name="helloPartnerLink" 
+         partnerLinkType="test:HelloPartnerLinkType" 
+         myRole="me" />
+   </partnerLinks>
+    
+   <variables>
+     <variable name="myVar" messageType="test:HelloMessage"/>
+     <variable name="tmpVar" type="xsd:string"/>
+   </variables>
+
+   <sequence>
+       <receive
+          name="start"
+          partnerLink="helloPartnerLink"
+          portType="test:HelloPortType"
+          operation="hello"
+          variable="myVar"
+          createInstance="yes"/>
+
+      <assign name="assign1">
+          <copy>
+              <from variable="myVar" part="TestPart"/>
+              <to variable="tmpVar"/>
+          </copy>
+          <copy>
+              <from>concat($tmpVar,' World')</from>
+              <to variable="myVar" part="TestPart"/>
+          </copy>
+      </assign>
+       <reply name="end"  
+              partnerLink="helloPartnerLink"
+              portType="test:HelloPortType" 
+              operation="hello"
+              variable="myVar"/>
+   </sequence>
+</process>

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/deploy.xml
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/deploy.xml?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/deploy.xml (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/deploy.xml Thu Aug 21 15:43:11 2008
@@ -0,0 +1,39 @@
+<!--
+  ~ 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.
+  -->
+<deploy xmlns="http://www.apache.org/ode/schemas/dd/2007/03"
+	xmlns:pns="http://ode/bpel/unit-test" 
+	xmlns:wns="http://ode/bpel/unit-test.wsdl">
+
+	<process name="pns:HelloWorld1">
+		<active>true</active>
+		<provide partnerLink="helloPartnerLink">
+			<service name="wns:HelloService" port="HelloPort"/>
+			<enableSharing/>
+		</provide>
+	</process>
+
+	<process name="pns:HelloWorld2">
+		<active>true</active>
+		<provide partnerLink="helloPartnerLink">
+			<service name="wns:HelloService" port="HelloPort"/>
+			<enableSharing/>
+		</provide>
+	</process>
+	
+</deploy>

Added: ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/test.properties
URL: http://svn.apache.org/viewvc/ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/test.properties?rev=687887&view=auto
==============================================================================
--- ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/test.properties (added)
+++ ode/branches/APACHE_ODE_1.X/bpel-test/src/test/resources/bpel/2.0/TestPubSubOutOfProc/test.properties Thu Aug 21 15:43:11 2008
@@ -0,0 +1,23 @@
+#
+#    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.
+#
+
+namespace=http://ode/bpel/unit-test.wsdl
+service=HelloService
+operation=hello
+request1=<message><TestPart><content>Hello</content></TestPart></message>
+response1=.*Hello World.*
+