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 2006/07/28 18:39:48 UTC

svn commit: r426609 [1/2] - in /incubator/ode/trunk: bpel-runtime/src/main/java/org/apache/ode/bpel/engine/ bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ bpel-runtime/src/test/java/org/apache/ode/bpel/runtime/ jacob/src/main/java/org/apache/o...

Author: mriou
Date: Fri Jul 28 09:39:45 2006
New Revision: 426609

URL: http://svn.apache.org/viewvc?rev=426609&view=rev
Log:
Changing Jacob lexicon as defined in http://mail-archives.apache.org/mod_mbox/incubator-ode-dev/200606.mbox/%3ccbb700270606231230r2aff6b45xcf0be879ad4c6d65@mail.gmail.com%3e

Added:
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelJacobRunnable.java
      - copied, changed from r426596, incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelAbstraction.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/ChannelListener.java
      - copied, changed from r426596, incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/ML.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobObject.java
      - copied, changed from r426596, incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JavaClosure.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobRunnable.java
      - copied, changed from r426596, incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/Abstraction.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Continuation.java
      - copied, changed from r426596, incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Reaction.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/ExecutionQueue.java
      - copied, changed from r426596, incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Soup.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/ExecutionQueueObject.java
      - copied, changed from r426596, incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/SoupObject.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/vpu/ExecutionQueueImpl.java
      - copied, changed from r426596, incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/vpu/FastSoupImpl.java
Removed:
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelAbstraction.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/Abstraction.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JavaClosure.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/ML.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Reaction.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Soup.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/SoupObject.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/vpu/FastSoupImpl.java
Modified:
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ACTIVITY.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ACTIVITYGUARD.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/COMPENSATE.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/COMPENSATIONHANDLER_.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EH_ALARM.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EH_EVENT.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EMPTY.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/FLOW.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/FOREACH.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/INVOKE.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ORDEREDCOMPENSATOR.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/PICK.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/PROCESS.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/SCOPE.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/SEQUENCE.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/TERMINATE.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/WAIT.java
    incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/WHILE.java
    incubator/ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/runtime/CoreBpelTest.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobThread.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/Sequence.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/cell/CELL_.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/eratosthenes/Sieve.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/synch/SynchPrinter.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Comm.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommChannel.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommGroup.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommRecv.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/vpu/JacobMessages.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/vpu/JacobVPU.java
    incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/vpu/Statistics.java
    incubator/ode/trunk/jacob/src/test/java/org/apache/ode/jacob/SynchTest.java
    incubator/ode/trunk/jacob/src/test/java/org/apache/ode/jacob/examples/cell/JacobCellTest.java

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java Fri Jul 28 09:39:45 2006
@@ -18,19 +18,28 @@
  */
 package org.apache.ode.bpel.engine;
 
-import org.apache.ode.jacob.Abstraction;
-import org.apache.ode.jacob.vpu.FastSoupImpl;
-import org.apache.ode.jacob.vpu.JacobVPU;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.common.CorrelationKey;
 import org.apache.ode.bpel.common.FaultException;
 import org.apache.ode.bpel.common.ProcessState;
-import org.apache.ode.bpel.dao.*;
+import org.apache.ode.bpel.dao.CorrelationSetDAO;
+import org.apache.ode.bpel.dao.CorrelatorDAO;
+import org.apache.ode.bpel.dao.MessageDAO;
+import org.apache.ode.bpel.dao.MessageExchangeDAO;
+import org.apache.ode.bpel.dao.PartnerLinkDAO;
+import org.apache.ode.bpel.dao.ProcessDAO;
+import org.apache.ode.bpel.dao.ProcessInstanceDAO;
+import org.apache.ode.bpel.dao.ScopeDAO;
+import org.apache.ode.bpel.dao.XmlDataDAO;
+import org.apache.ode.bpel.epr.MutableEndpoint;
+import org.apache.ode.bpel.epr.WSAEndpoint;
 import org.apache.ode.bpel.evt.CorrelationSetWriteEvent;
 import org.apache.ode.bpel.evt.ProcessCompletionEvent;
 import org.apache.ode.bpel.evt.ProcessInstanceEvent;
 import org.apache.ode.bpel.evt.ProcessInstanceStateChangeEvent;
-import org.apache.ode.bpel.evt.ProcessTerminationEvent;
 import org.apache.ode.bpel.evt.ProcessMessageExchangeEvent;
+import org.apache.ode.bpel.evt.ProcessTerminationEvent;
 import org.apache.ode.bpel.iapi.BpelEngineException;
 import org.apache.ode.bpel.iapi.ContextException;
 import org.apache.ode.bpel.iapi.EndpointReference;
@@ -38,12 +47,12 @@
 import org.apache.ode.bpel.iapi.MessageExchange.FailureType;
 import org.apache.ode.bpel.iapi.MessageExchange.MessageExchangePattern;
 import org.apache.ode.bpel.o.OMessageVarType;
+import org.apache.ode.bpel.o.OMessageVarType.Part;
 import org.apache.ode.bpel.o.OPartnerLink;
 import org.apache.ode.bpel.o.OProcess;
 import org.apache.ode.bpel.o.OScope;
 import org.apache.ode.bpel.o.OVarType;
-import org.apache.ode.bpel.o.OMessageVarType.Part;
-import org.apache.ode.bpel.runtime.BpelAbstraction;
+import org.apache.ode.bpel.runtime.BpelJacobRunnable;
 import org.apache.ode.bpel.runtime.BpelRuntimeContext;
 import org.apache.ode.bpel.runtime.CorrelationSetInstance;
 import org.apache.ode.bpel.runtime.ExpressionLanguageRuntimeRegistry;
@@ -55,33 +64,27 @@
 import org.apache.ode.bpel.runtime.channels.InvokeResponseChannel;
 import org.apache.ode.bpel.runtime.channels.PickResponseChannel;
 import org.apache.ode.bpel.runtime.channels.TimerResponseChannel;
-import org.apache.ode.bpel.epr.MutableEndpoint;
-import org.apache.ode.bpel.epr.WSDL11Endpoint;
-import org.apache.ode.bpel.epr.WSAEndpoint;
+import org.apache.ode.jacob.JacobRunnable;
+import org.apache.ode.jacob.vpu.ExecutionQueueImpl;
+import org.apache.ode.jacob.vpu.JacobVPU;
 import org.apache.ode.utils.DOMUtils;
-import org.apache.ode.utils.ObjectPrinter;
-import org.apache.ode.utils.Namespaces;
 import org.apache.ode.utils.GUID;
+import org.apache.ode.utils.Namespaces;
+import org.apache.ode.utils.ObjectPrinter;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
 
+import javax.wsdl.Operation;
+import javax.xml.namespace.QName;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Date;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
-import java.util.HashMap;
-
-import javax.wsdl.Operation;
-import javax.xml.namespace.QName;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import sun.rmi.transport.Endpoint;
 
 class BpelRuntimeContextImpl implements BpelRuntimeContext {
 
@@ -100,8 +103,8 @@
   /** JACOB VPU */
   protected JacobVPU vpu;
 
-  /** JACOB Soup (state) */
-  protected FastSoupImpl soup;
+  /** JACOB ExecutionQueue (state) */
+  protected ExecutionQueueImpl soup;
 
   private MyRoleMessageExchangeImpl _instantiatingMessageExchange;
 
@@ -867,7 +870,7 @@
     _outstandingRequests.associate(responsechannel, mex.getMessageExchangeId());
 
     final String mexId = mex.getMessageExchangeId();
-    vpu.inject(new Abstraction() {
+    vpu.inject(new JacobRunnable() {
       private static final long serialVersionUID = 3168964409165899533L;
 
       public void self() {
@@ -889,7 +892,7 @@
       return;
     }
 
-    vpu.inject(new Abstraction() {
+    vpu.inject(new JacobRunnable() {
       private static final long serialVersionUID = -7767141033611036745L;
 
       public void self() {
@@ -908,7 +911,7 @@
     _dao.getProcess().removeRoutes(id, _dao);
     _outstandingRequests.cancel(id);
 
-    vpu.inject(new Abstraction() {
+    vpu.inject(new JacobRunnable() {
       private static final long serialVersionUID = 6157913683737696396L;
 
       public void self() {
@@ -925,7 +928,7 @@
 
   void invocationResponse(final String mexid, final String responseChannelId) {
 
-    vpu.inject(new BpelAbstraction() {
+    vpu.inject(new BpelJacobRunnable() {
       private static final long serialVersionUID = -1095444335740879981L;
 
       public void self() {
@@ -984,7 +987,7 @@
   private void initVPU() {
     vpu = new JacobVPU();
     vpu.registerExtension(BpelRuntimeContext.class, this);
-    soup = new FastSoupImpl(null);
+    soup = new ExecutionQueueImpl(null);
     soup.setReplacementMap(_bpelProcess._replacementMap);
     vpu.setContext(soup);
   }

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ACTIVITY.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ACTIVITY.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ACTIVITY.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ACTIVITY.java Fri Jul 28 09:39:45 2006
@@ -31,7 +31,7 @@
 /**
  * Base template for activities.
  */
-abstract class ACTIVITY extends BpelAbstraction {
+abstract class ACTIVITY extends BpelJacobRunnable {
   protected ActivityInfo _self;
 
   /**

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ACTIVITYGUARD.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ACTIVITYGUARD.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ACTIVITYGUARD.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ACTIVITYGUARD.java Fri Jul 28 09:39:45 2006
@@ -18,8 +18,8 @@
  */
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.ML;
-import org.apache.ode.jacob.SynchChannel;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.common.FaultException;
 import org.apache.ode.bpel.evt.ActivityEnabledEvent;
 import org.apache.ode.bpel.evt.ActivityExecEndEvent;
@@ -29,12 +29,19 @@
 import org.apache.ode.bpel.o.OExpression;
 import org.apache.ode.bpel.o.OLink;
 import org.apache.ode.bpel.o.OScope;
-import org.apache.ode.bpel.runtime.channels.*;
-
-import java.util.*;
+import org.apache.ode.bpel.runtime.channels.FaultData;
+import org.apache.ode.bpel.runtime.channels.LinkStatusChannelListener;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannel;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannelListener;
+import org.apache.ode.bpel.runtime.channels.TerminationChannelListener;
+import org.apache.ode.jacob.ChannelListener;
+import org.apache.ode.jacob.SynchChannel;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
 
 class ACTIVITYGUARD extends ACTIVITY {
 	private static final long serialVersionUID = 1L;
@@ -80,8 +87,8 @@
         dpe(_oactivity);
       }
     } else /* don't know all our links statuses */ {
-      Set<ML> mlset = new HashSet<ML>();
-      mlset.add(new TerminationML(_self.self) {
+      Set<ChannelListener> mlset = new HashSet<ChannelListener>();
+      mlset.add(new TerminationChannelListener(_self.self) {
         private static final long serialVersionUID = 5094153128476008961L;
 
         public void terminate() {
@@ -94,7 +101,7 @@
       });
       for (Iterator<OLink> i = _oactivity.targetLinks.iterator();i.hasNext();) {
         final OLink link = i.next();
-        mlset.add(new LinkStatusML(_linkFrame.resolve(link).sub) {
+        mlset.add(new LinkStatusChannelListener(_linkFrame.resolve(link).sub) {
         private static final long serialVersionUID = 1024137371118887935L;
 
         public void linkStatus(boolean value) {
@@ -161,7 +168,7 @@
    * {@link ParentScopeChannel#completed(org.apache.ode.bpel.runtime.channels.FaultData, java.util.Set<org.apache.ode.bpel.runtime.CompensationHandler>)}
    * call, to evaluate transition conditions before returning to the parent.
    */
-  private class TCONDINTERCEPT extends BpelAbstraction {
+  private class TCONDINTERCEPT extends BpelJacobRunnable {
     private static final long serialVersionUID = 4014873396828400441L;
     ParentScopeChannel _in;
 
@@ -170,7 +177,7 @@
     }
 
     public void self() {
-      object(new ParentScopeML(_in) {
+      object(new ParentScopeChannelListener(_in) {
         private static final long serialVersionUID = 2667359535900385952L;
 
         public void compensate(OScope scope, SynchChannel ret) {

Copied: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelJacobRunnable.java (from r426596, incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelAbstraction.java)
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelJacobRunnable.java?p2=incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelJacobRunnable.java&p1=incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelAbstraction.java&r1=426596&r2=426609&rev=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelAbstraction.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelJacobRunnable.java Fri Jul 28 09:39:45 2006
@@ -18,16 +18,16 @@
  */
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.Abstraction;
-import org.apache.ode.jacob.vpu.JacobVPU;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.common.CorrelationKey;
 import org.apache.ode.bpel.common.FaultException;
 import org.apache.ode.bpel.o.OBase;
 import org.apache.ode.bpel.o.OProcess;
 import org.apache.ode.bpel.o.OVarType;
 import org.apache.ode.bpel.runtime.channels.FaultData;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.apache.ode.jacob.JacobRunnable;
+import org.apache.ode.jacob.vpu.JacobVPU;
 import org.w3c.dom.Element;
 
 import javax.xml.namespace.QName;
@@ -40,8 +40,8 @@
  * Created on Jan 12, 2004 at 5:41:27 PM.
  * @author Maciej Szefler
  */
-public abstract class BpelAbstraction extends Abstraction {
-  private static final Log __log = LogFactory.getLog(BpelAbstraction.class);
+public abstract class BpelJacobRunnable extends JacobRunnable {
+  private static final Log __log = LogFactory.getLog(BpelJacobRunnable.class);
 
   protected BpelRuntimeContext getBpelRuntimeContext() {
     BpelRuntimeContext nativeApi = (BpelRuntimeContext) JacobVPU.activeJacobThread().getExtension(BpelRuntimeContext.class);
@@ -66,7 +66,7 @@
   }
   
 
-  protected Abstraction createChild(ActivityInfo childInfo, ScopeFrame scopeFrame, LinkFrame linkFrame) {
+  protected JacobRunnable createChild(ActivityInfo childInfo, ScopeFrame scopeFrame, LinkFrame linkFrame) {
     return new ACTIVITYGUARD(childInfo, scopeFrame, linkFrame);
   }
 

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/COMPENSATE.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/COMPENSATE.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/COMPENSATE.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/COMPENSATE.java Fri Jul 28 09:39:45 2006
@@ -18,10 +18,10 @@
  */
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.SynchChannel;
-import org.apache.ode.jacob.SynchML;
 import org.apache.ode.bpel.o.OCompensate;
 import org.apache.ode.bpel.o.OScope;
+import org.apache.ode.jacob.SynchChannel;
+import org.apache.ode.jacob.SynchChannelListener;
 
 
 /**
@@ -40,7 +40,7 @@
     OScope scopeToCompensate = _ocompact.compensatedScope;
     SynchChannel sc = newChannel(SynchChannel.class);
     _self.parent.compensate(scopeToCompensate,sc);
-    object(new SynchML(sc) {
+    object(new SynchChannelListener(sc) {
     private static final long serialVersionUID = 3763991229748926216L;
 
     public void ret() {

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/COMPENSATIONHANDLER_.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/COMPENSATIONHANDLER_.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/COMPENSATIONHANDLER_.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/COMPENSATIONHANDLER_.java Fri Jul 28 09:39:45 2006
@@ -18,11 +18,15 @@
  */
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.SynchChannel;
 import org.apache.ode.bpel.evt.CompensationHandlerRegistered;
 import org.apache.ode.bpel.evt.ScopeEvent;
 import org.apache.ode.bpel.o.OScope;
-import org.apache.ode.bpel.runtime.channels.*;
+import org.apache.ode.bpel.runtime.channels.CompensationChannelListener;
+import org.apache.ode.bpel.runtime.channels.FaultData;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannel;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannelListener;
+import org.apache.ode.bpel.runtime.channels.TerminationChannel;
+import org.apache.ode.jacob.SynchChannel;
 
 import java.util.Iterator;
 import java.util.Set;
@@ -30,7 +34,7 @@
 /**
  * A scope that has completed succesfully, and may possibly have a compensation handler.
  */
-class COMPENSATIONHANDLER_ extends BpelAbstraction {
+class COMPENSATIONHANDLER_ extends BpelJacobRunnable {
 	private static final long serialVersionUID = 1L;
 	private CompensationHandler _self;
   private Set<CompensationHandler> _completedChildren;
@@ -42,7 +46,7 @@
 
   public void self() {
     sendEvent(new CompensationHandlerRegistered());
-    object(new CompensationML(_self.compChannel) {
+    object(new CompensationChannelListener(_self.compChannel) {
     private static final long serialVersionUID = -477602498730810094L;
 
     public void forget() {
@@ -69,7 +73,7 @@
          // Create the compensation handler scope.
          instance(new SCOPE(ai,compHandlerScopeFrame, new LinkFrame(null)));
 
-         object(new ParentScopeML(ai.parent) {
+         object(new ParentScopeChannelListener(ai.parent) {
         private static final long serialVersionUID = 8044120498580711546L;
 
         public void compensate(OScope scope, SynchChannel ret) {

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EH_ALARM.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EH_ALARM.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EH_ALARM.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EH_ALARM.java Fri Jul 28 09:39:45 2006
@@ -18,13 +18,21 @@
  */
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.SynchChannel;
 import org.apache.ode.bpel.common.FaultException;
 import org.apache.ode.bpel.explang.EvaluationContext;
 import org.apache.ode.bpel.explang.EvaluationException;
 import org.apache.ode.bpel.o.OEventHandler;
 import org.apache.ode.bpel.o.OScope;
-import org.apache.ode.bpel.runtime.channels.*;
+import org.apache.ode.bpel.runtime.channels.EventHandlerControlChannel;
+import org.apache.ode.bpel.runtime.channels.EventHandlerControlChannelListener;
+import org.apache.ode.bpel.runtime.channels.FaultData;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannel;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannelListener;
+import org.apache.ode.bpel.runtime.channels.TerminationChannel;
+import org.apache.ode.bpel.runtime.channels.TerminationChannelListener;
+import org.apache.ode.bpel.runtime.channels.TimerResponseChannel;
+import org.apache.ode.bpel.runtime.channels.TimerResponseChannelListener;
+import org.apache.ode.jacob.SynchChannel;
 
 import java.util.Calendar;
 import java.util.HashSet;
@@ -36,7 +44,7 @@
  * "soft" termination (aka stopping) to deal with the case when the scope that owns
  * the event handler completes naturally.
  */
-class EH_ALARM extends BpelAbstraction {
+class EH_ALARM extends BpelJacobRunnable {
 	private static final long serialVersionUID = 1L;
 
 	private ParentScopeChannel _psc;
@@ -98,7 +106,7 @@
    * elapsed time. This template also monitors the termination and event-control channels
    * for requests from parent.
    */
-  private class WAIT extends BpelAbstraction {
+  private class WAIT extends BpelJacobRunnable {
     private static final long serialVersionUID = -1426724996925898213L;
     Calendar _alarm;
 
@@ -116,7 +124,7 @@
       if (now.before(_alarm)) {
         TimerResponseChannel trc = newChannel(TimerResponseChannel.class);
         getBpelRuntimeContext().registerTimer(trc,_alarm.getTime());
-        object(false,new TimerResponseML(trc){
+        object(false,new TimerResponseChannelListener(trc){
         private static final long serialVersionUID = 1110683632756756017L;
 
         public void onTimeout() {
@@ -127,14 +135,14 @@
           public void onCancel() {
             _psc.completed(null, _comps);
           }
-        }.or(new EventHandlerControlML(_cc) {
+        }.or(new EventHandlerControlChannelListener(_cc) {
         private static final long serialVersionUID = -7750428941445331236L;
 
         public void stop() {
             _psc.completed(null, _comps);
           }
 
-        }.or(new TerminationML(_tc) {
+        }.or(new TerminationChannelListener(_tc) {
         private static final long serialVersionUID = 6100105997983514609L;
 
         public void terminate() {
@@ -152,7 +160,7 @@
   /**
    * Snipped that fires the alarm activity.
    */
-  private class FIRE extends BpelAbstraction {
+  private class FIRE extends BpelJacobRunnable {
     private static final long serialVersionUID = -7261315204412433250L;
 
     public void self() {
@@ -168,7 +176,7 @@
   /**
    * Snippet that is used to monitor a running activity.
    */
-  private class ACTIVE extends BpelAbstraction {
+  private class ACTIVE extends BpelJacobRunnable {
     private static final long serialVersionUID = -2166253425722769701L;
 
     private ActivityInfo _activity;
@@ -181,7 +189,7 @@
     }
 
     public void self() {
-      object(false,new ParentScopeML(_activity.parent){
+      object(false,new ParentScopeChannelListener(_activity.parent){
         private static final long serialVersionUID = -3357030137175178040L;
 
         public void compensate(OScope scope, SynchChannel ret) {
@@ -207,7 +215,7 @@
           }
         }
 
-      }.or(new EventHandlerControlML(_cc) {
+      }.or(new EventHandlerControlChannelListener(_cc) {
         private static final long serialVersionUID = -3873619538789039424L;
 
         public void stop() {
@@ -215,7 +223,7 @@
           instance(ACTIVE.this);
         }
 
-      }.or(new TerminationML(_tc) {
+      }.or(new TerminationChannelListener(_tc) {
         private static final long serialVersionUID = -4566956567870652885L;
 
         public void terminate() {

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EH_EVENT.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EH_EVENT.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EH_EVENT.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EH_EVENT.java Fri Jul 28 09:39:45 2006
@@ -18,15 +18,23 @@
  */
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.ML;
-import org.apache.ode.jacob.SynchChannel;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.common.CorrelationKey;
 import org.apache.ode.bpel.common.FaultException;
 import org.apache.ode.bpel.o.OEventHandler;
 import org.apache.ode.bpel.o.OScope;
-import org.apache.ode.bpel.runtime.channels.*;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.apache.ode.bpel.runtime.channels.EventHandlerControlChannel;
+import org.apache.ode.bpel.runtime.channels.EventHandlerControlChannelListener;
+import org.apache.ode.bpel.runtime.channels.FaultData;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannel;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannelListener;
+import org.apache.ode.bpel.runtime.channels.PickResponseChannel;
+import org.apache.ode.bpel.runtime.channels.PickResponseChannelListener;
+import org.apache.ode.bpel.runtime.channels.TerminationChannel;
+import org.apache.ode.bpel.runtime.channels.TerminationChannelListener;
+import org.apache.ode.jacob.ChannelListener;
+import org.apache.ode.jacob.SynchChannel;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
 
@@ -36,7 +44,7 @@
 /**
  * Message event handler.
  */
-class EH_EVENT extends BpelAbstraction {
+class EH_EVENT extends BpelJacobRunnable {
 
 	private static final long serialVersionUID = 1L;
 
@@ -93,12 +101,12 @@
    * Template that does the actual selection interaction with the runtime system, and
    * then waits on the pick response channel.
    */
-  class SELECT extends BpelAbstraction {
+  class SELECT extends BpelJacobRunnable {
 
 		private static final long serialVersionUID = 1L;
 
 		/**
-     * @see org.apache.ode.jacob.Abstraction#self()
+     * @see org.apache.ode.jacob.JacobRunnable#self()
      */
     public void self() {
       Selector selector;
@@ -139,7 +147,7 @@
   /**
    * Template that represents the waiting for a pick response.
    */
-  private class WAITING extends BpelAbstraction {
+  private class WAITING extends BpelJacobRunnable {
 		private static final long serialVersionUID = 1L;
 		private PickResponseChannel _pickResponseChannel;
 
@@ -150,10 +158,10 @@
     public void self() {
 
       if (!_active.isEmpty() || _pickResponseChannel != null) {
-        HashSet<ML> mlset = new HashSet<ML>();
+        HashSet<ChannelListener> mlset = new HashSet<ChannelListener>();
 
         if (!_terminated) {
-          mlset.add(new TerminationML(_tc) {
+          mlset.add(new TerminationChannelListener(_tc) {
             private static final long serialVersionUID = 7666910462948788042L;
 
             public void terminate() {
@@ -168,7 +176,7 @@
         }
 
         if (!_stopped) {
-          mlset.add(new EventHandlerControlML(_ehc) {
+          mlset.add(new EventHandlerControlChannelListener(_ehc) {
             private static final long serialVersionUID = -1050788954724647970L;
 
             public void stop() {
@@ -182,7 +190,7 @@
         }
 
         for (final ActivityInfo ai : _active) {
-          mlset.add(new ParentScopeML(ai.parent) {
+          mlset.add(new ParentScopeChannelListener(ai.parent) {
             private static final long serialVersionUID = 5341207762415360982L;
 
             public void compensate(OScope scope, SynchChannel ret) {
@@ -204,7 +212,7 @@
         }
 
         if (_pickResponseChannel != null)
-          mlset.add(new PickResponseML(_pickResponseChannel) {
+          mlset.add(new PickResponseChannelListener(_pickResponseChannel) {
             private static final long serialVersionUID = -4929999153478677288L;
 
 

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EMPTY.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EMPTY.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EMPTY.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EMPTY.java Fri Jul 28 09:39:45 2006
@@ -23,7 +23,7 @@
 
 
 /**
- * Abstraction that performs the work of the <code>empty</code> activity.
+ * JacobRunnable that performs the work of the <code>empty</code> activity.
  */
 class EMPTY extends ACTIVITY {
 	private static final long serialVersionUID = 1L;

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/FLOW.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/FLOW.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/FLOW.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/FLOW.java Fri Jul 28 09:39:45 2006
@@ -18,13 +18,18 @@
  */
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.ML;
-import org.apache.ode.jacob.SynchChannel;
 import org.apache.ode.bpel.o.OActivity;
 import org.apache.ode.bpel.o.OFlow;
 import org.apache.ode.bpel.o.OLink;
 import org.apache.ode.bpel.o.OScope;
-import org.apache.ode.bpel.runtime.channels.*;
+import org.apache.ode.bpel.runtime.channels.FaultData;
+import org.apache.ode.bpel.runtime.channels.LinkStatusChannel;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannel;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannelListener;
+import org.apache.ode.bpel.runtime.channels.TerminationChannel;
+import org.apache.ode.bpel.runtime.channels.TerminationChannelListener;
+import org.apache.ode.jacob.ChannelListener;
+import org.apache.ode.jacob.SynchChannel;
 import org.apache.ode.utils.stl.FilterIterator;
 import org.apache.ode.utils.stl.MemberOfFunction;
 
@@ -67,7 +72,7 @@
   }
 
 
-  private class ACTIVE extends BpelAbstraction {
+  private class ACTIVE extends BpelJacobRunnable {
     private static final long serialVersionUID = -8494641460279049245L;
     private FaultData _fault;
     private HashSet<CompensationHandler> _compensations = new HashSet<CompensationHandler>();
@@ -75,8 +80,8 @@
     public void self() {
       Iterator<ChildInfo> active = active();
       if (active.hasNext()) {
-        Set<ML> mlSet = new HashSet<ML>();
-        mlSet.add(new TerminationML(_self.self) {
+        Set<ChannelListener> mlSet = new HashSet<ChannelListener>();
+        mlSet.add(new TerminationChannelListener(_self.self) {
         private static final long serialVersionUID = 2554750258974084466L;
 
         public void terminate() {
@@ -90,7 +95,7 @@
 
         for (;active.hasNext();) {
           final ChildInfo child = active.next();
-          mlSet.add(new ParentScopeML(child.activity.parent) {
+          mlSet.add(new ParentScopeChannelListener(child.activity.parent) {
             private static final long serialVersionUID = -8027205709169238172L;
 
             public void completed(FaultData faultData, Set<CompensationHandler> compensations) {

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/FOREACH.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/FOREACH.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/FOREACH.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/FOREACH.java Fri Jul 28 09:39:45 2006
@@ -19,20 +19,24 @@
 
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.ML;
-import org.apache.ode.jacob.SynchChannel;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.common.FaultException;
 import org.apache.ode.bpel.elang.xpath10.o.OXPath10Expression;
 import org.apache.ode.bpel.explang.EvaluationException;
 import org.apache.ode.bpel.o.OExpression;
 import org.apache.ode.bpel.o.OForEach;
 import org.apache.ode.bpel.o.OScope;
-import org.apache.ode.bpel.runtime.channels.*;
+import org.apache.ode.bpel.runtime.channels.FaultData;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannel;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannelListener;
+import org.apache.ode.bpel.runtime.channels.TerminationChannel;
+import org.apache.ode.bpel.runtime.channels.TerminationChannelListener;
+import org.apache.ode.jacob.ChannelListener;
+import org.apache.ode.jacob.SynchChannel;
 import org.apache.ode.utils.DOMUtils;
 import org.apache.ode.utils.stl.FilterIterator;
 import org.apache.ode.utils.stl.MemberOfFunction;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 import org.w3c.dom.Document;
 import org.w3c.dom.Node;
 
@@ -94,7 +98,7 @@
     }
   }
 
-  private class ACTIVE extends BpelAbstraction {
+  private class ACTIVE extends BpelJacobRunnable {
     private static final long serialVersionUID = -5642862698981385732L;
 
     private FaultData _fault;
@@ -105,8 +109,8 @@
       // Continuing as long as a child is active
       if (active().hasNext()) {
 
-        Set<ML> mlSet = new HashSet<ML>();
-        mlSet.add(new TerminationML(_self.self) {
+        Set<ChannelListener> mlSet = new HashSet<ChannelListener>();
+        mlSet.add(new TerminationChannelListener(_self.self) {
           private static final long serialVersionUID = 2554750257484084466L;
 
           public void terminate() {
@@ -120,7 +124,7 @@
         for (;active.hasNext();) {
           // Checking out our children
           final ChildInfo child = active.next();
-          mlSet.add(new ParentScopeML(child.activity.parent) {
+          mlSet.add(new ParentScopeChannelListener(child.activity.parent) {
             private static final long serialVersionUID = -8027205709961438172L;
 
             public void compensate(OScope scope, SynchChannel ret) {

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/INVOKE.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/INVOKE.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/INVOKE.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/INVOKE.java Fri Jul 28 09:39:45 2006
@@ -23,16 +23,15 @@
 import org.apache.ode.bpel.o.OScope;
 import org.apache.ode.bpel.runtime.channels.FaultData;
 import org.apache.ode.bpel.runtime.channels.InvokeResponseChannel;
-import org.apache.ode.bpel.runtime.channels.InvokeResponseML;
+import org.apache.ode.bpel.runtime.channels.InvokeResponseChannelListener;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
 
 import javax.xml.namespace.QName;
 import java.util.Collection;
-import java.util.Iterator;
 
 /**
- * Abstraction that performs the work of the <code>invoke</code> activity.
+ * JacobRunnable that performs the work of the <code>invoke</code> activity.
  */
 public class INVOKE extends ACTIVITY {
   private static final long serialVersionUID = 992248281026821783L;
@@ -63,7 +62,7 @@
         getBpelRuntimeContext().invoke(
             _scopeFrame.resolve(_oinvoke.partnerLink),
             _oinvoke.operation,
-            outboundMsg, 
+            outboundMsg,
             null);
 
         _self.parent.completed(faultData, CompensationHandler.emptySet());
@@ -75,10 +74,10 @@
 
         final String mexId = getBpelRuntimeContext().invoke(
             _scopeFrame.resolve(_oinvoke.partnerLink), _oinvoke.operation,
-            outboundMsg, 
+            outboundMsg,
             invokeResponseChannel);
 
-        object(new InvokeResponseML(invokeResponseChannel) {
+        object(new InvokeResponseChannelListener(invokeResponseChannel) {
           private static final long serialVersionUID = 4496880438819196765L;
 
           public void onResponse() {
@@ -143,7 +142,7 @@
   }
 
   private Element setupOutbound(OInvoke oinvoke,
-      Collection<OScope.CorrelationSet> outboundInitiations)
+                                Collection<OScope.CorrelationSet> outboundInitiations)
       throws FaultException {
     if (outboundInitiations.size() > 0) {
       for (OScope.CorrelationSet c : outboundInitiations) {

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ORDEREDCOMPENSATOR.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ORDEREDCOMPENSATOR.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ORDEREDCOMPENSATOR.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ORDEREDCOMPENSATOR.java Fri Jul 28 09:39:45 2006
@@ -18,18 +18,17 @@
  */
 package org.apache.ode.bpel.runtime;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.ode.jacob.SynchChannel;
-import org.apache.ode.jacob.SynchML;
+import org.apache.ode.jacob.SynchChannelListener;
 
 import java.util.List;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 /**
  * Serially activates a list of compensations in order.
  */
-class ORDEREDCOMPENSATOR extends BpelAbstraction  {
+class ORDEREDCOMPENSATOR extends BpelJacobRunnable  {
   private static final long serialVersionUID = -3181661355085428370L;
 
   private static final Log __log = LogFactory.getLog(ORDEREDCOMPENSATOR.class);
@@ -50,7 +49,7 @@
       SynchChannel r = newChannel(SynchChannel.class);
       CompensationHandler cdata = _compensations.remove(0);
       cdata.compChannel.compensate(r);
-      object(new SynchML(r) {
+      object(new SynchChannelListener(r) {
         private static final long serialVersionUID = 7173916663479205420L;
 
         public void ret() {

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/PICK.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/PICK.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/PICK.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/PICK.java Fri Jul 28 09:39:45 2006
@@ -18,6 +18,8 @@
  */
 package org.apache.ode.bpel.runtime;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.common.CorrelationKey;
 import org.apache.ode.bpel.common.FaultException;
 import org.apache.ode.bpel.explang.EvaluationException;
@@ -25,27 +27,24 @@
 import org.apache.ode.bpel.o.OScope;
 import org.apache.ode.bpel.runtime.channels.FaultData;
 import org.apache.ode.bpel.runtime.channels.PickResponseChannel;
-import org.apache.ode.bpel.runtime.channels.PickResponseML;
-import org.apache.ode.bpel.runtime.channels.TerminationML;
-import org.apache.ode.utils.xsd.Duration;
+import org.apache.ode.bpel.runtime.channels.PickResponseChannelListener;
+import org.apache.ode.bpel.runtime.channels.TerminationChannelListener;
 import org.apache.ode.utils.DOMUtils;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.apache.ode.utils.xsd.Duration;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
 
 import java.util.Calendar;
 import java.util.Date;
-import java.util.Iterator;
 
 
 /**
  * Template for the BPEL <code>pick</code> activity.
  */
 class PICK extends ACTIVITY {
-	private static final long serialVersionUID = 1L;
+  private static final long serialVersionUID = 1L;
 
-	private static final Log __log = LogFactory.getLog(PICK.class);
+  private static final Log __log = LogFactory.getLog(PICK.class);
 
   private OPickReceive _opick;
 
@@ -60,7 +59,7 @@
 
 
   /**
-   * @see org.apache.ode.jacob.Abstraction#self()
+   * @see org.apache.ode.jacob.JacobRunnable#self()
    */
   public void self() {
     PickResponseChannel pickResponseChannel = newChannel(PickResponseChannel.class);
@@ -139,10 +138,10 @@
   }
 
 
-  private class WAITING extends BpelAbstraction {
-		private static final long serialVersionUID = 1L;
-		
-		private PickResponseChannel _pickResponseChannel;
+  private class WAITING extends BpelJacobRunnable {
+    private static final long serialVersionUID = 1L;
+
+    private PickResponseChannel _pickResponseChannel;
 
     private WAITING(PickResponseChannel pickResponseChannel) {
       this._pickResponseChannel = pickResponseChannel;
@@ -150,7 +149,7 @@
 
     public void self() {
 
-      object(false, new PickResponseML(_pickResponseChannel) {
+      object(false, new PickResponseChannelListener(_pickResponseChannel) {
         private static final long serialVersionUID = -8237296827418738011L;
 
         public void onRequestRcvd(int selectorIdx, String mexId) {
@@ -221,7 +220,7 @@
           _self.parent.completed(null, CompensationHandler.emptySet());
         }
 
-      }.or(new TerminationML(_self.self) {
+      }.or(new TerminationChannelListener(_self.self) {
         private static final long serialVersionUID = 4399496341785922396L;
 
         public void terminate() {

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/PROCESS.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/PROCESS.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/PROCESS.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/PROCESS.java Fri Jul 28 09:39:45 2006
@@ -18,18 +18,18 @@
  */
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.SynchChannel;
 import org.apache.ode.bpel.evt.ProcessInstanceStartedEvent;
 import org.apache.ode.bpel.o.OProcess;
 import org.apache.ode.bpel.o.OScope;
 import org.apache.ode.bpel.runtime.channels.FaultData;
 import org.apache.ode.bpel.runtime.channels.ParentScopeChannel;
-import org.apache.ode.bpel.runtime.channels.ParentScopeML;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannelListener;
 import org.apache.ode.bpel.runtime.channels.TerminationChannel;
+import org.apache.ode.jacob.SynchChannel;
 
 import java.util.Set;
 
-public class PROCESS extends BpelAbstraction {
+public class PROCESS extends BpelJacobRunnable {
 	private static final long serialVersionUID = 1L;
 	private OProcess _process;
 
@@ -52,7 +52,7 @@
     ScopeFrame processFrame = new ScopeFrame(_process.procesScope, scopeInstanceId, null, null);
     instance(new SCOPE(child, processFrame, new LinkFrame(null)));
 
-    object(new ParentScopeML(child.parent) {
+    object(new ParentScopeChannelListener(child.parent) {
     private static final long serialVersionUID = -8564969578471906493L;
 
     public void compensate(OScope scope, SynchChannel ret) {

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/SCOPE.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/SCOPE.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/SCOPE.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/SCOPE.java Fri Jul 28 09:39:45 2006
@@ -18,20 +18,37 @@
  */
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.ML;
-import org.apache.ode.jacob.SynchChannel;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.evt.ScopeFaultEvent;
 import org.apache.ode.bpel.evt.ScopeStartEvent;
-import org.apache.ode.bpel.o.*;
-import org.apache.ode.bpel.runtime.channels.*;
-
-import java.io.Serializable;
-import java.util.*;
+import org.apache.ode.bpel.o.OBase;
+import org.apache.ode.bpel.o.OCatch;
+import org.apache.ode.bpel.o.OElementVarType;
+import org.apache.ode.bpel.o.OEventHandler;
+import org.apache.ode.bpel.o.OFaultHandler;
+import org.apache.ode.bpel.o.OLink;
+import org.apache.ode.bpel.o.OMessageVarType;
+import org.apache.ode.bpel.o.OScope;
+import org.apache.ode.bpel.o.OVarType;
+import org.apache.ode.bpel.runtime.channels.CompensationChannel;
+import org.apache.ode.bpel.runtime.channels.EventHandlerControlChannel;
+import org.apache.ode.bpel.runtime.channels.FaultData;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannel;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannelListener;
+import org.apache.ode.bpel.runtime.channels.TerminationChannel;
+import org.apache.ode.bpel.runtime.channels.TerminationChannelListener;
+import org.apache.ode.jacob.ChannelListener;
+import org.apache.ode.jacob.SynchChannel;
 
 import javax.xml.namespace.QName;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
 
 /**
  * An active scope.
@@ -121,10 +138,10 @@
 
     public void self() {
       if (_child != null || !_eventHandlers.isEmpty()) {
-        HashSet<ML> mlSet = new HashSet<ML>();
+        HashSet<ChannelListener> mlSet = new HashSet<ChannelListener>();
 
         // Listen to messages from our parent.
-        mlSet.add(new TerminationML(_self.self) {
+        mlSet.add(new TerminationChannelListener(_self.self) {
         private static final long serialVersionUID = 1913414844895865116L;
 
         public void terminate() {
@@ -146,7 +163,7 @@
 
         // Handle messages from the child if it is still alive
         if (_child != null)
-          mlSet.add(new ParentScopeML(_child.parent) {
+          mlSet.add(new ParentScopeChannelListener(_child.parent) {
             private static final long serialVersionUID = -6934246487304813033L;
 
 
@@ -187,7 +204,7 @@
         for (Iterator<EventHandlerInfo> i = _eventHandlers.iterator();i.hasNext();) {
           final EventHandlerInfo ehi = i.next();
 
-          mlSet.add(new ParentScopeML(ehi.psc) {
+          mlSet.add(new ParentScopeChannelListener(ehi.psc) {
             private static final long serialVersionUID = -4694721357537858221L;
 
             public void compensate(OScope scope, SynchChannel ret) {
@@ -294,7 +311,7 @@
             // Create the fault handler scope.
             instance(new SCOPE(faultHandlerActivity,faultHandlerScopeFrame, SCOPE.this._linkFrame));
 
-            object(new ParentScopeML(faultHandlerActivity.parent) {
+            object(new ParentScopeChannelListener(faultHandlerActivity.parent) {
             private static final long serialVersionUID = -6009078124717125270L;
 
             public void compensate(OScope scope, SynchChannel ret) {

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/SEQUENCE.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/SEQUENCE.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/SEQUENCE.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/SEQUENCE.java Fri Jul 28 09:39:45 2006
@@ -18,13 +18,22 @@
  */
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.SynchChannel;
 import org.apache.ode.bpel.o.OActivity;
 import org.apache.ode.bpel.o.OScope;
 import org.apache.ode.bpel.o.OSequence;
-import org.apache.ode.bpel.runtime.channels.*;
+import org.apache.ode.bpel.runtime.channels.FaultData;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannel;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannelListener;
+import org.apache.ode.bpel.runtime.channels.TerminationChannel;
+import org.apache.ode.bpel.runtime.channels.TerminationChannelListener;
+import org.apache.ode.jacob.SynchChannel;
 
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
 
 /**
  * Implementation of the BPEL &lt;sequence&gt; activity.
@@ -56,7 +65,7 @@
     instance(new ACTIVE(child));
   }
 
-  private class ACTIVE extends BpelAbstraction {
+  private class ACTIVE extends BpelJacobRunnable {
     private static final long serialVersionUID = -2663862698981385732L;
     private ActivityInfo _child;
     private boolean _terminateRequested = false;
@@ -66,7 +75,7 @@
     }
 
     public void self() {
-      object(false, new TerminationML(_self.self) {
+      object(false, new TerminationChannelListener(_self.self) {
         private static final long serialVersionUID = -2680515407515637639L;
 
         public void terminate() {
@@ -78,7 +87,7 @@
           _terminateRequested = true;
           instance(ACTIVE.this);
         }
-      }.or(new ParentScopeML(_child.parent) {
+      }.or(new ParentScopeChannelListener(_child.parent) {
         private static final long serialVersionUID = 7195562310281985971L;
 
         public void compensate(OScope scope, SynchChannel ret) {

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/TERMINATE.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/TERMINATE.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/TERMINATE.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/TERMINATE.java Fri Jul 28 09:39:45 2006
@@ -19,7 +19,7 @@
 package org.apache.ode.bpel.runtime;
 
 /**
- * Abstraction that performs the work of the <code>empty</code> activity.
+ * JacobRunnable that performs the work of the <code>empty</code> activity.
  */
 class TERMINATE extends ACTIVITY {
 

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/WAIT.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/WAIT.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/WAIT.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/WAIT.java Fri Jul 28 09:39:45 2006
@@ -18,24 +18,23 @@
  */
 package org.apache.ode.bpel.runtime;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.common.FaultException;
 import org.apache.ode.bpel.explang.EvaluationContext;
 import org.apache.ode.bpel.explang.EvaluationException;
 import org.apache.ode.bpel.o.OWait;
-import org.apache.ode.bpel.runtime.channels.TerminationML;
+import org.apache.ode.bpel.runtime.channels.TerminationChannelListener;
 import org.apache.ode.bpel.runtime.channels.TimerResponseChannel;
-import org.apache.ode.bpel.runtime.channels.TimerResponseML;
+import org.apache.ode.bpel.runtime.channels.TimerResponseChannelListener;
 import org.apache.ode.utils.xsd.Duration;
 
 import java.util.Calendar;
 import java.util.Date;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 
 /**
- * Abstraction that performs the work of the <code>&lt;wait&gt;</code> activity.
+ * JacobRunnable that performs the work of the <code>&lt;wait&gt;</code> activity.
  */
 class WAIT extends ACTIVITY {
 	private static final long serialVersionUID = 1L;
@@ -67,7 +66,7 @@
 	    final TimerResponseChannel timerChannel = newChannel(TimerResponseChannel.class);
 	    getBpelRuntimeContext().registerTimer(timerChannel, dueDate);
 	    
-	    object(false, new TimerResponseML(timerChannel){
+	    object(false, new TimerResponseChannelListener(timerChannel){
          private static final long serialVersionUID = 3120518305645437327L;
 
          public void onTimeout() {
@@ -77,12 +76,12 @@
         public void onCancel() {
           _self.parent.completed(null, CompensationHandler.emptySet());
         }
-      }.or(new TerminationML(_self.self) {
+      }.or(new TerminationChannelListener(_self.self) {
         private static final long serialVersionUID = -2791243270691333946L;
 
         public void terminate() {
           _self.parent.completed(null, CompensationHandler.emptySet());
-          object(new TimerResponseML(timerChannel) {
+          object(new TimerResponseChannelListener(timerChannel) {
             private static final long serialVersionUID = 677746737897792929L;
 
             public void onTimeout() {

Modified: incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/WHILE.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/WHILE.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/WHILE.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/WHILE.java Fri Jul 28 09:39:45 2006
@@ -18,19 +18,22 @@
  */
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.SynchChannel;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.common.FaultException;
 import org.apache.ode.bpel.explang.EvaluationException;
 import org.apache.ode.bpel.o.OScope;
 import org.apache.ode.bpel.o.OWhile;
-import org.apache.ode.bpel.runtime.channels.*;
+import org.apache.ode.bpel.runtime.channels.FaultData;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannel;
+import org.apache.ode.bpel.runtime.channels.ParentScopeChannelListener;
+import org.apache.ode.bpel.runtime.channels.TerminationChannel;
+import org.apache.ode.bpel.runtime.channels.TerminationChannelListener;
+import org.apache.ode.jacob.SynchChannel;
 
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 /**
  * BPEL &lt;while&gt; activity
  */
@@ -99,7 +102,7 @@
     }
   }
 
-  private class WAITER extends BpelAbstraction {
+  private class WAITER extends BpelJacobRunnable {
     private static final long serialVersionUID = -7645042174027252066L;
     private ActivityInfo _child;
     private boolean _terminated;
@@ -109,7 +112,7 @@
     }
 
     public void self() {
-      object(false, new TerminationML(_self.self) {
+      object(false, new TerminationChannelListener(_self.self) {
         private static final long serialVersionUID = -5471984635653784051L;
 
         public void terminate() {
@@ -117,7 +120,7 @@
           replication(_child.self).terminate();
           instance(WAITER.this);
         }
-      }.or(new ParentScopeML(_child.parent) {
+      }.or(new ParentScopeChannelListener(_child.parent) {
         private static final long serialVersionUID = 3907167240907524405L;
 
         public void compensate(OScope scope, SynchChannel ret) {

Modified: incubator/ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/runtime/CoreBpelTest.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/runtime/CoreBpelTest.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/runtime/CoreBpelTest.java (original)
+++ incubator/ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/runtime/CoreBpelTest.java Fri Jul 28 09:39:45 2006
@@ -18,30 +18,36 @@
  */
 package org.apache.ode.bpel.runtime;
 
-import org.apache.ode.jacob.vpu.FastSoupImpl;
-import org.apache.ode.jacob.vpu.JacobVPU;
+import junit.framework.TestCase;
 import org.apache.ode.bpel.common.CorrelationKey;
 import org.apache.ode.bpel.common.FaultException;
 import org.apache.ode.bpel.evt.ProcessInstanceEvent;
-import org.apache.ode.bpel.o.*;
+import org.apache.ode.bpel.o.OCatch;
+import org.apache.ode.bpel.o.OEmpty;
+import org.apache.ode.bpel.o.OFaultHandler;
+import org.apache.ode.bpel.o.OFlow;
+import org.apache.ode.bpel.o.OMessageVarType;
 import org.apache.ode.bpel.o.OMessageVarType.Part;
+import org.apache.ode.bpel.o.OPartnerLink;
+import org.apache.ode.bpel.o.OProcess;
+import org.apache.ode.bpel.o.OScope;
+import org.apache.ode.bpel.o.OSequence;
+import org.apache.ode.bpel.o.OThrow;
 import org.apache.ode.bpel.runtime.channels.FaultData;
 import org.apache.ode.bpel.runtime.channels.InvokeResponseChannel;
 import org.apache.ode.bpel.runtime.channels.PickResponseChannel;
 import org.apache.ode.bpel.runtime.channels.TimerResponseChannel;
+import org.apache.ode.jacob.vpu.ExecutionQueueImpl;
+import org.apache.ode.jacob.vpu.JacobVPU;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
 
+import javax.wsdl.Operation;
+import javax.xml.namespace.QName;
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
-import java.util.Date;
 import java.util.Collection;
-
-import javax.wsdl.Operation;
-import javax.xml.namespace.QName;
-
-import junit.framework.TestCase;
-
-import org.w3c.dom.Element;
-import org.w3c.dom.Node;
+import java.util.Date;
 
 /**
  * Test core BPEL processing capabilities.
@@ -50,7 +56,7 @@
   private boolean _completedOk;
   private boolean _terminate;
   private FaultData _fault;
-  private FastSoupImpl _soup;
+  private ExecutionQueueImpl _soup;
   private JacobVPU _vpu;
   private Long _pid;
   private long _seq;
@@ -59,7 +65,7 @@
     _completedOk= false;
     _terminate = false;
     _fault = null;
-    _soup = new FastSoupImpl(CoreBpelTest.class.getClassLoader());
+    _soup = new ExecutionQueueImpl(CoreBpelTest.class.getClassLoader());
     _vpu = new JacobVPU(_soup);
     _vpu.registerExtension(BpelRuntimeContext.class, this);
     _pid = new Long(19355);

Copied: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/ChannelListener.java (from r426596, incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/ML.java)
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/ChannelListener.java?p2=incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/ChannelListener.java&p1=incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/ML.java&r1=426596&r2=426609&rev=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/ML.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/ChannelListener.java Fri Jul 28 09:39:45 2006
@@ -18,33 +18,33 @@
  */
 package org.apache.ode.jacob;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
 import java.lang.reflect.Method;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 
 /**
  * Base-class for method-list objects. Method-lists objects should extends
  * this class <em>and</em> implement one <code>Channel</code> interface.
  */
-public abstract class ML<CT extends Channel> extends JavaClosure {
-  private static Log __log = LogFactory.getLog(ML.class);
+public abstract class ChannelListener<CT extends Channel> extends JacobObject {
+  private static Log __log = LogFactory.getLog(ChannelListener.class);
 
   private transient Set<Method> _implementedMethods;
   private transient CT _channel;
 
-  protected ML(CT channel)
+  protected ChannelListener(CT channel)
         throws IllegalStateException {
 
     if (this.getClass()
                   .getSuperclass()
-                  .getSuperclass() != ML.class) {
-      throw new IllegalStateException("Inheritence in ML classes not allowed!");
+                  .getSuperclass() != ChannelListener.class) {
+      throw new IllegalStateException("Inheritence in ChannelListener classes not allowed!");
     }
 
     if (channel == null)
@@ -57,15 +57,15 @@
 
   public void setChannel(CT channel) { _channel = channel; }
 
-  public Set<ML> or(ML other) {
-    HashSet<ML> retval = new HashSet<ML>();
+  public Set<ChannelListener> or(ChannelListener other) {
+    HashSet<ChannelListener> retval = new HashSet<ChannelListener>();
     retval.add(this);
     retval.add(other);
     return retval;
   }
 
-  public Set<ML> or(Set<ML> other) {
-    HashSet<ML> retval = new HashSet<ML>(other);
+  public Set<ChannelListener> or(Set<ChannelListener> other) {
+    HashSet<ChannelListener> retval = new HashSet<ChannelListener>(other);
     retval.add(this);
     return retval;
   }

Copied: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobObject.java (from r426596, incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JavaClosure.java)
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobObject.java?p2=incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobObject.java&p1=incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JavaClosure.java&r1=426596&r2=426609&rev=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JavaClosure.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobObject.java Fri Jul 28 09:39:45 2006
@@ -18,6 +18,8 @@
  */
 package org.apache.ode.jacob;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.ode.jacob.vpu.JacobVPU;
 
 import java.io.Serializable;
@@ -25,16 +27,13 @@
 import java.util.Iterator;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 
 /**
  * Base class for constructs which rely on a Java method body to represent
  * some aspect of the process.
  */
-public abstract class JavaClosure implements Serializable {
-  private static final Log __log = LogFactory.getLog(JavaClosure.class);
+public abstract class JacobObject implements Serializable {
+  private static final Log __log = LogFactory.getLog(JacobObject.class);
 
   public abstract Set getImplementedMethods();
 
@@ -88,7 +87,7 @@
    *
    * @param concretion the concretion of a process template
    */
-  protected void instance(Abstraction concretion) {
+  protected void instance(JacobRunnable concretion) {
     JacobVPU.activeJacobThread().instance(concretion);
   }
 
@@ -124,31 +123,31 @@
   }
 
   /**
-   * Object; the Java code "object(x, ML)" is equivalent to <code>x ?
-   * ML</code> in the process algebra.
+   * Object; the Java code "object(x, ChannelListener)" is equivalent to <code>x ?
+   * ChannelListener</code> in the process algebra.
    *
    * @param methodList method list for the communication reduction
    *
    * @see JacobThread#object
    */
-  protected <T extends Channel> T object(ML<T> methodList) {
+  protected <T extends Channel> T object(ChannelListener<T> methodList) {
     JacobVPU.activeJacobThread().object(false, methodList);
     return methodList.getChannel();
   }
 
-  protected void object(boolean replication, ML methodList) {
+  protected void object(boolean replication, ChannelListener methodList) {
     JacobVPU.activeJacobThread().object(replication, methodList);
   }
 
-  protected void object(boolean replication, ML[] methodLists) {
+  protected void object(boolean replication, ChannelListener[] methodLists) {
     JacobVPU.activeJacobThread().object(replication, methodLists);
   }
 
-  protected void object(boolean replication, Set<ML> methodLists) {
-    JacobVPU.activeJacobThread().object(replication, methodLists.toArray(new ML[methodLists.size()]));
+  protected void object(boolean replication, Set<ChannelListener> methodLists) {
+    JacobVPU.activeJacobThread().object(replication, methodLists.toArray(new ChannelListener[methodLists.size()]));
   }
 
-  protected  <T extends Channel> T replication(ML<T> methodList) {
+  protected  <T extends Channel> T replication(ChannelListener<T> methodList) {
     JacobVPU.activeJacobThread().object(true, methodList);
     return methodList.getChannel();
   }

Copied: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobRunnable.java (from r426596, incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/Abstraction.java)
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobRunnable.java?p2=incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobRunnable.java&p1=incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/Abstraction.java&r1=426596&r2=426609&rev=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/Abstraction.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobRunnable.java Fri Jul 28 09:39:45 2006
@@ -19,15 +19,14 @@
 
 package org.apache.ode.jacob;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.ode.utils.ArrayUtils;
 
 import java.lang.reflect.Method;
 import java.util.Collections;
 import java.util.Set;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-
 
 /**
  * Base class for process abstractions. An abstraction is a parameterized
@@ -39,7 +38,7 @@
  * would be represented by the following Java class:
  * <code>
  * <pre>
- * public class Cell extends Abstraction {
+ * public class Cell extends JacobRunnable {
  *   private CellChannel s;
  *   private Object v;
  *   public Cell(CellChannel s, Object v) {
@@ -47,7 +46,7 @@
  *     this.v = v;
  *   }
  *   public void self() {
- *     object(new CellML(s) { read(...) {...}
+ *     object(new CellChannelListener(s) { read(...) {...}
  *                            write(...) {...} } );
  *   }
  * }
@@ -70,13 +69,13 @@
  *
  * @author Maciej Szefler <a href="mailto:mbs@fivesight.com" />
  */
-public abstract class Abstraction extends JavaClosure {
-  private static final Log __log = LogFactory.getLog(Abstraction.class);
+public abstract class JacobRunnable extends JacobObject {
+  private static final Log __log = LogFactory.getLog(JacobRunnable.class);
 
   private static final Set<Method> IMPLEMENTED_METHODS;
   static {
     try {
-      IMPLEMENTED_METHODS = Collections.singleton(Abstraction.class.getMethod("self", ArrayUtils.EMPTY_CLASS_ARRAY));
+      IMPLEMENTED_METHODS = Collections.singleton(JacobRunnable.class.getMethod("self", ArrayUtils.EMPTY_CLASS_ARRAY));
     } catch (NoSuchMethodException e) {
       throw new AssertionError(e);
     }
@@ -112,7 +111,7 @@
   }
 
   /**
-   * @see JavaClosure#log
+   * @see JacobObject#log
    */
   protected Log log() {
     return __log;

Modified: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobThread.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobThread.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobThread.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/JacobThread.java Fri Jul 28 09:39:45 2006
@@ -63,7 +63,7 @@
    *
    * @param concretion new concretion of some abstraction
    */
-  public void instance(Abstraction concretion);
+  public void instance(JacobRunnable concretion);
 
   /**
    * Send a message (object invocation). This method shouldn't really be used
@@ -96,10 +96,10 @@
    * 
    * <p>
    * With respect to process terms, the Java expression <code>object(false, x,
-   * ML)</code> corresponds to the process term <code> x ? { ML }</code>; if
+   * ChannelListener)</code> corresponds to the process term <code> x ? { ChannelListener }</code>; if
    * in the same expression the initial <code>replicate</code> parameter were
    * instead set to <code>true</code>, corresponding term would be <code> ! x
-   * ? { ML }</code>.
+   * ? { ChannelListener }</code>.
    * </p>
    *
    * @param replicate if set the a replication operator is present
@@ -108,10 +108,10 @@
    * @throws IllegalArgumentException if the method list does not match the
    *         channel kind
    */
-  public void object(boolean replicate, ML methodList)
+  public void object(boolean replicate, ChannelListener methodList)
               throws IllegalArgumentException;
 
 
-  public void object(boolean reaplicate, ML[] methodLists)
+  public void object(boolean reaplicate, ChannelListener[] methodLists)
               throws IllegalArgumentException;
 }

Modified: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/Sequence.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/Sequence.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/Sequence.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/Sequence.java Fri Jul 28 09:39:45 2006
@@ -18,7 +18,7 @@
  */
 package org.apache.ode.jacob;
 
-public abstract class Sequence extends Abstraction {
+public abstract class Sequence extends JacobRunnable {
   private int _size;
   private int _step;
   private SynchChannel _done;
@@ -43,7 +43,7 @@
       }
     } else {
       SynchChannel r = newChannel(SynchChannel.class);
-      object(new SynchML(r) {
+      object(new SynchChannelListener(r) {
           public void ret() {
             ++_step;
             instance(Sequence.this);
@@ -53,5 +53,5 @@
     }
   }
 
-  protected abstract Abstraction reduce(int n, SynchChannel r);
+  protected abstract JacobRunnable reduce(int n, SynchChannel r);
 }

Modified: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/cell/CELL_.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/cell/CELL_.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/cell/CELL_.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/cell/CELL_.java Fri Jul 28 09:39:45 2006
@@ -18,7 +18,7 @@
  */
 package org.apache.ode.jacob.examples.cell;
 
-import org.apache.ode.jacob.Abstraction;
+import org.apache.ode.jacob.JacobRunnable;
 import org.apache.ode.jacob.Val;
 
 
@@ -28,7 +28,7 @@
  * read(r) = { Cell(self, val) | r ! val(val) } & write(newVal) = {
  * Cell(self, newVal) } ] </code>
  */
-public class CELL_<T> extends Abstraction {
+public class CELL_<T> extends JacobRunnable {
   private CellChannel _self;
   private T _val;
 
@@ -40,7 +40,7 @@
   public void self() {
     // INSTANTIATION{Cell(self,val)}
     // ==> self ? [ read(r)={...} & write(newVal)={...} ]
-    object(new CellML(_self) {
+    object(new CellChannelListener(_self) {
         public void read(Val r) {
           // COMMUNICATION{x & [read... & ... ] | x ! read}
           // ==> Cell(self, val) ...

Modified: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/eratosthenes/Sieve.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/eratosthenes/Sieve.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/eratosthenes/Sieve.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/eratosthenes/Sieve.java Fri Jul 28 09:39:45 2006
@@ -18,10 +18,10 @@
  */
 package org.apache.ode.jacob.examples.eratosthenes;
 
-import org.apache.ode.jacob.Abstraction;
+import org.apache.ode.jacob.JacobRunnable;
 import org.apache.ode.jacob.SynchChannel;
-import org.apache.ode.jacob.SynchML;
-import org.apache.ode.jacob.vpu.FastSoupImpl;
+import org.apache.ode.jacob.SynchChannelListener;
+import org.apache.ode.jacob.vpu.ExecutionQueueImpl;
 import org.apache.ode.jacob.vpu.JacobVPU;
 
 /**
@@ -36,7 +36,7 @@
  * 
  * @author Maciej Szefler <a href="mailto:mbs@fivesight.com">mbs</a>
  */
-public class Sieve extends Abstraction {
+public class Sieve extends JacobRunnable {
   private static int _cnt = 0;
   private static int _last = 0;
 
@@ -56,7 +56,7 @@
    *  Counter(out, n) := out.val(n) | Counter(out, n+1)
    * </em></pre>
    */
-  private static class Counter extends Abstraction {
+  private static class Counter extends JacobRunnable {
     private NaturalNumberStreamChannel _out;
     private int _n;
 
@@ -66,7 +66,7 @@
     }
 
     public void self() {
-      _out.val(_n, object(new SynchML(newChannel(SynchChannel.class)) {
+      _out.val(_n, object(new SynchChannelListener(newChannel(SynchChannel.class)) {
         public void ret() {
           instance(new Counter(_out, _n+1));
         }
@@ -84,7 +84,7 @@
    *
    *
    */
-  private static final class Head extends Abstraction {
+  private static final class Head extends JacobRunnable {
     NaturalNumberStreamChannel _in;
     NaturalNumberStreamChannel _primes;
 
@@ -94,9 +94,9 @@
     }
 
     public void self() {
-      object(new NaturalNumberStreamML(_in) {
+      object(new NaturalNumberStreamChannelListener(_in) {
         public void val(final int n, final SynchChannel ret) {
-          _primes.val(n, object(new SynchML(newChannel(SynchChannel.class)) {
+          _primes.val(n, object(new SynchChannelListener(newChannel(SynchChannel.class)) {
             public void ret() {
               NaturalNumberStreamChannel x = newChannel(NaturalNumberStreamChannel.class);
               instance(new PrimeFilter(n, _in, x));
@@ -109,13 +109,13 @@
     }
   }
 
-  private static final class Print extends Abstraction {
+  private static final class Print extends JacobRunnable {
     private NaturalNumberStreamChannel _in;
     public Print(NaturalNumberStreamChannel in) {
       _in = in;
     }
     public void self() {
-      object(true, new NaturalNumberStreamML(_in){
+      object(true, new NaturalNumberStreamChannelListener(_in){
         public void val(int n, SynchChannel ret) {
           _cnt ++;
           _last = n;
@@ -134,7 +134,7 @@
    *     ! in ? [val(n)={ if(n mod prime <> 0) out.val(n) }
    * </em></prime>
    */
-  private static class PrimeFilter extends Abstraction {
+  private static class PrimeFilter extends JacobRunnable {
     private int _prime;
     private NaturalNumberStreamChannel _in;
     private NaturalNumberStreamChannel _out;
@@ -145,10 +145,10 @@
       _out = out;
     }
     public void self() {
-       object(true, new NaturalNumberStreamML(_in) {
+       object(true, new NaturalNumberStreamChannelListener(_in) {
          public void val(int n, final SynchChannel ret) {
            if (n % _prime != 0)
-             _out.val(n, object(new SynchML(newChannel(SynchChannel.class)) {
+             _out.val(n, object(new SynchChannelListener(newChannel(SynchChannel.class)) {
                public void ret() {
                  ret.ret();
                }
@@ -170,7 +170,7 @@
     } else {
       int request = Integer.parseInt(args[0]);
       JacobVPU vpu = new JacobVPU();
-      vpu.setContext(new FastSoupImpl(null));
+      vpu.setContext(new ExecutionQueueImpl(null));
       vpu.inject(new Sieve());
       while (_cnt != request) {
         vpu.execute();

Modified: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/synch/SynchPrinter.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/synch/SynchPrinter.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/synch/SynchPrinter.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/examples/synch/SynchPrinter.java Fri Jul 28 09:39:45 2006
@@ -18,10 +18,10 @@
  */
 package org.apache.ode.jacob.examples.synch;
 
-import org.apache.ode.jacob.Abstraction;
+import org.apache.ode.jacob.JacobRunnable;
 import org.apache.ode.jacob.SynchChannel;
-import org.apache.ode.jacob.SynchML;
-import org.apache.ode.jacob.vpu.FastSoupImpl;
+import org.apache.ode.jacob.SynchChannelListener;
+import org.apache.ode.jacob.vpu.ExecutionQueueImpl;
 import org.apache.ode.jacob.vpu.JacobVPU;
 
 /**
@@ -32,13 +32,13 @@
  */
 public class SynchPrinter  {
 
-  public static final class SystemPrinter extends Abstraction {
+  public static final class SystemPrinter extends JacobRunnable {
     private SynchPrintChannel self;
     public SystemPrinter(SynchPrintChannel self) {
       this.self = self;
     }
     public void self() {
-      object(true, new SynchPrintML(self) {
+      object(true, new SynchPrintChannelListener(self) {
         public SynchChannel print(String msg) {
           System.out.println(msg);
           return null;
@@ -47,15 +47,15 @@
     }
   }
 
-  public static final class Tester extends Abstraction {
+  public static final class Tester extends JacobRunnable {
     public void self() {
       final SynchPrintChannel p = newChannel(SynchPrintChannel.class);
       instance(new SystemPrinter(p));
-      object(new SynchML(p.print("1")) {
+      object(new SynchChannelListener(p.print("1")) {
         public void ret() {
-          object(new SynchML(p.print("2")) {
+          object(new SynchChannelListener(p.print("2")) {
             public void ret() {
-              object(new SynchML(p.print("3")) {
+              object(new SynchChannelListener(p.print("3")) {
                 public void ret() {
                 }
               });
@@ -68,7 +68,7 @@
 
   public static void main(String args[]) {
     JacobVPU vpu = new JacobVPU();
-    vpu.setContext(new FastSoupImpl(null));
+    vpu.setContext(new ExecutionQueueImpl(null));
     vpu.inject(new Tester());
     while (vpu.execute()) {
       // run

Modified: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Comm.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Comm.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Comm.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Comm.java Fri Jul 28 09:39:45 2006
@@ -26,7 +26,7 @@
  * 
  * @author Maciej Szefler <a href="mailto:mbs@fivesight.com">mbs</a>
  */
-public abstract  class Comm extends SoupObject {
+public abstract  class Comm extends ExecutionQueueObject {
   private CommChannel _channel;
   private CommGroup _group;
 

Modified: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommChannel.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommChannel.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommChannel.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommChannel.java Fri Jul 28 09:39:45 2006
@@ -34,7 +34,7 @@
  * 
  */
 
-public class CommChannel extends SoupObject {
+public class CommChannel extends ExecutionQueueObject {
 
   private Class _type;
 

Modified: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommGroup.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommGroup.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommGroup.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommGroup.java Fri Jul 28 09:39:45 2006
@@ -34,7 +34,7 @@
  * 
  */
 
-public class CommGroup extends SoupObject {
+public class CommGroup extends ExecutionQueueObject {
 
   boolean _isReplicated;
   List<Comm> _comms = new ArrayList<Comm>();

Modified: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommRecv.java
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommRecv.java?rev=426609&r1=426608&r2=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommRecv.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/CommRecv.java Fri Jul 28 09:39:45 2006
@@ -18,7 +18,7 @@
  */
 package org.apache.ode.jacob.soup;
 
-import org.apache.ode.jacob.ML;
+import org.apache.ode.jacob.ChannelListener;
 
 /**
  * Persistent store representation of an object (i.e. channel read) waiting
@@ -29,11 +29,11 @@
  * whether the read is of a replicated variety.
  */
 public class CommRecv extends Comm {
-  private ML _continuation;
+  private ChannelListener _continuation;
 
   protected CommRecv() {}
 
-  public CommRecv(CommChannel chnl,  ML continuation) {
+  public CommRecv(CommChannel chnl,  ChannelListener continuation) {
     super(null, chnl);
     _continuation = continuation;
   }
@@ -42,12 +42,12 @@
    * Get the continuation for this object (channel read). The continuation is
    * what happens after a message is matched to the object. It is up to the
    * JACOB VPU to determine what is placed here, but it will generally
-   * consist of some serialized representation of an appropriate ML object
-   * (see {@link ML}.
+   * consist of some serialized representation of an appropriate ChannelListener object
+   * (see {@link ChannelListener}.
    *
    * @return byte array representing the serialized form of the continuation
    */
-  public ML getContinuation() { return _continuation ;}
+  public ChannelListener getContinuation() { return _continuation ;}
 
   public String toString() {
     StringBuffer buf = new StringBuffer(getChannel().toString());

Copied: incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Continuation.java (from r426596, incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Reaction.java)
URL: http://svn.apache.org/viewvc/incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Continuation.java?p2=incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Continuation.java&p1=incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Reaction.java&r1=426596&r2=426609&rev=426609&view=diff
==============================================================================
--- incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Reaction.java (original)
+++ incubator/ode/trunk/jacob/src/main/java/org/apache/ode/jacob/soup/Continuation.java Fri Jul 28 09:39:45 2006
@@ -18,29 +18,29 @@
  */
 package org.apache.ode.jacob.soup;
 
-import java.lang.reflect.Method;
-
-import org.apache.ode.jacob.JavaClosure;
+import org.apache.ode.jacob.JacobObject;
 import org.apache.ode.utils.ObjectPrinter;
 
+import java.lang.reflect.Method;
+
 /**
  * DOCUMENTME.
  * <p>Created on Feb 16, 2004 at 9:23:40 PM.</p>
  * 
  * @author Maciej Szefler <a href="mailto:mbs@fivesight.com">mbs</a>
  */
-public class Reaction extends SoupObject {
-  private JavaClosure _closure;
+public class Continuation extends ExecutionQueueObject {
+  private JacobObject _closure;
   private Method _method;
   private Object[] _args;
 
-  public Reaction(JavaClosure target, Method method, Object[] args) {
+  public Continuation(JacobObject target, Method method, Object[] args) {
     _closure = target;
     _method = method;
     _args = args;
   }
 
-  public JavaClosure getClosure() {
+  public JacobObject getClosure() {
     return _closure;
   }