You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ode.apache.org by va...@apache.org on 2007/10/16 00:48:36 UTC

svn commit: r584955 - in /ode/trunk: axis2/src/main/java/org/apache/ode/axis2/ bpel-api/src/main/java/org/apache/ode/bpel/eapi/ bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/ bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/api/ bpel...

Author: vanto
Date: Mon Oct 15 15:48:30 2007
New Revision: 584955

URL: http://svn.apache.org/viewvc?rev=584955&view=rev
Log:
IN PROGRESS - issue ODE-160: ExtensionActivity and ExtensionAssignOperation: Runtime support 
http://issues.apache.org/jira/browse/ODE-160

o Support for extension validators during compiling phase have been added.
o Extension API has been refactored and divided into a runtime and a compile time part.

Added:
    ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/api/ExtensionValidator.java
    ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensibleElement.java
    ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/
    ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/AbstractExtensionBundle.java
      - copied, changed from r583317, ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/AbstractExtensionBundle.java
    ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/AbstractExtensionOperation.java
    ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/ExtensionContext.java
      - copied, changed from r583317, ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/ExtensionContext.java
    ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/ExtensionOperation.java
      - copied, changed from r583317, ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/ExtensionOperation.java
    ode/trunk/bpel-test/src/main/java/org/apache/ode/test/MockExtensionContext.java
Removed:
    ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/AbstractExtensionBundle.java
    ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/ExtensionContext.java
    ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/ExtensionOperation.java
Modified:
    ode/trunk/axis2/src/main/java/org/apache/ode/axis2/ODEServer.java
    ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/AssignGenerator.java
    ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/BpelC.java
    ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/BpelCompiler.java
    ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/ExtensionActivtityGenerator.java
    ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/api/CompilerContext.java
    ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensionActivity.java
    ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensionAssignOperation.java
    ode/trunk/bpel-compiler/src/test/java/org/apache/ode/bpel/compiler/XPathTest.java
    ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java
    ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelServerImpl.java
    ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/Contexts.java
    ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ASSIGN.java
    ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelRuntimeContext.java
    ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EXTENSIONACTIVITY.java
    ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ExtensionContextImpl.java
    ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/elang/xpath20/runtime/MockCompilerContext.java
    ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/runtime/CoreBpelTest.java
    ode/trunk/bpel-store/src/main/java/org/apache/ode/store/DeploymentUnitDir.java
    ode/trunk/bpel-store/src/main/java/org/apache/ode/store/ProcessStoreImpl.java
    ode/trunk/bpel-test/src/main/java/org/apache/ode/test/BPELTestAbstract.java
    ode/trunk/bpel-test/src/test/java/org/apache/ode/test/ExtensibilityTest.java
    ode/trunk/bpel-test/src/test/resources/bpel/2.0/TestExtensionActivity/ExtensionActivity.bpel
    ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/ExtensionContextWrapper.java
    ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/JSExtensionBundle.java
    ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/JSExtensionOperation.java
    ode/trunk/extensions/e4x/src/test/java/org/apache/ode/extension/e4x/JSOperationTest.java

Modified: ode/trunk/axis2/src/main/java/org/apache/ode/axis2/ODEServer.java
URL: http://svn.apache.org/viewvc/ode/trunk/axis2/src/main/java/org/apache/ode/axis2/ODEServer.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/axis2/src/main/java/org/apache/ode/axis2/ODEServer.java (original)
+++ ode/trunk/axis2/src/main/java/org/apache/ode/axis2/ODEServer.java Mon Oct 15 15:48:30 2007
@@ -21,6 +21,8 @@
 
 import java.io.File;
 import java.io.FileNotFoundException;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.StringTokenizer;
 
 import javax.servlet.ServletConfig;
@@ -42,9 +44,9 @@
 import org.apache.ode.axis2.hooks.ODEMessageReceiver;
 import org.apache.ode.axis2.service.DeploymentWebService;
 import org.apache.ode.axis2.service.ManagementService;
+import org.apache.ode.bpel.compiler.api.ExtensionValidator;
 import org.apache.ode.bpel.connector.BpelServerConnector;
 import org.apache.ode.bpel.dao.BpelDAOConnectionFactory;
-import org.apache.ode.bpel.eapi.AbstractExtensionBundle;
 import org.apache.ode.bpel.engine.BpelServerImpl;
 import org.apache.ode.bpel.engine.CountLRUDehydrationPolicy;
 import org.apache.ode.bpel.evtproc.DebugBpelEventListener;
@@ -55,6 +57,7 @@
 import org.apache.ode.bpel.iapi.ProcessStoreListener;
 import org.apache.ode.bpel.iapi.Scheduler;
 import org.apache.ode.bpel.intercept.MessageExchangeInterceptor;
+import org.apache.ode.bpel.runtime.extension.AbstractExtensionBundle;
 import org.apache.ode.il.dbutil.Database;
 import org.apache.ode.scheduler.simple.JdbcDelegate;
 import org.apache.ode.scheduler.simple.SimpleScheduler;
@@ -512,18 +515,28 @@
     }
 
  	private void registerExtensionActivityBundles() {
-        String listenersStr = _odeConfig.getExtensionActivityBundles();
-        if (listenersStr != null) {
+        String extensionsStr = _odeConfig.getExtensionActivityBundles();
+        if (extensionsStr != null) {
+        	Map<QName, ExtensionValidator> validators = new HashMap<QName, ExtensionValidator>();
         	// TODO replace StringTokenizer by regex
-        	for (StringTokenizer tokenizer = new StringTokenizer(listenersStr, ",;"); tokenizer.hasMoreTokens();) {
+        	for (StringTokenizer tokenizer = new StringTokenizer(extensionsStr, ",;"); tokenizer.hasMoreTokens();) {
                 String bundleCN = tokenizer.nextToken();
                 try {
-                    _server.registerExtensionBundle((AbstractExtensionBundle) Class.forName(bundleCN).newInstance());
+                	// instantiate bundle
+                	AbstractExtensionBundle bundle = (AbstractExtensionBundle) Class.forName(bundleCN).newInstance();
+                	
+                	// register extension bundle (BPEL server)
+                	_server.registerExtensionBundle(bundle);
+                	
+                	//add validators
+                	validators.putAll(bundle.getExtensionValidators());
                 } catch (Exception e) {
                     __log.warn("Couldn't register the extension bundle " + bundleCN + ", the class couldn't be " +
                             "loaded properly.");
                 }
             }
+        	// register extension bundle (BPEL store)
+        	_store.setExtensionValidators(validators);
         }
     }
 

Modified: ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/AssignGenerator.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/AssignGenerator.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/AssignGenerator.java (original)
+++ ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/AssignGenerator.java Mon Oct 15 15:48:30 2007
@@ -23,6 +23,7 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.compiler.api.CompilationException;
+import org.apache.ode.bpel.compiler.api.ExtensionValidator;
 import org.apache.ode.bpel.compiler.bom.Activity;
 import org.apache.ode.bpel.compiler.bom.AssignActivity;
 import org.apache.ode.bpel.compiler.bom.Copy;
@@ -102,6 +103,10 @@
         			if (!_context.isExtensionDeclared(el.getNamespaceURI())) {
         				throw new CompilationException(__cmsgs.errUndeclaredExtensionAssignOperation().setSource(sop));
         			}
+        	        ExtensionValidator validator = _context.getExtensionValidator(DOMUtils.getElementQName(el));
+        	        if (validator != null) {
+        	        	validator.validate(sop);
+        	        }
         			oext.nestedElement = new SerializableElement(el);
             		oassign.operations.add(oext);
         		} catch (CompilationException ce) {

Modified: ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/BpelC.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/BpelC.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/BpelC.java (original)
+++ ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/BpelC.java Mon Oct 15 15:48:30 2007
@@ -18,11 +18,23 @@
  */
 package org.apache.ode.bpel.compiler;
 
+import java.io.BufferedOutputStream;
+import java.io.ByteArrayInputStream;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.Map;
+
+import javax.xml.namespace.QName;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.compiler.api.CompilationException;
 import org.apache.ode.bpel.compiler.api.CompilationMessage;
 import org.apache.ode.bpel.compiler.api.CompileListener;
+import org.apache.ode.bpel.compiler.api.ExtensionValidator;
 import org.apache.ode.bpel.compiler.api.SourceLocation;
 import org.apache.ode.bpel.compiler.bom.BpelObjectFactory;
 import org.apache.ode.bpel.compiler.bom.Process;
@@ -33,16 +45,6 @@
 import org.w3c.dom.Node;
 import org.xml.sax.InputSource;
 
-import javax.xml.namespace.QName;
-import java.io.BufferedOutputStream;
-import java.io.ByteArrayInputStream;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.URI;
-import java.util.Map;
-
 /**
  * <p>
  * Wrapper for {@link org.apache.ode.bpel.compiler.BpelCompiler} implementations,
@@ -64,6 +66,7 @@
     private ResourceFinder _wsdlFinder;
     private URI _bpel11wsdl;
     private Map<String,Object> _compileProperties;
+    private Map<QName, ExtensionValidator> _extensionValidators;
     private boolean _dryRun = false;
 
     public static BpelC newBpelCompiler() {
@@ -138,6 +141,15 @@
     }
 
     /**
+     * Registers extension validators to eventually validate the content of extensibility
+     * elements. 
+     * @param extensionValidators
+     */
+    public void setExtensionValidators(Map<QName, ExtensionValidator> extensionValidators) {
+        _extensionValidators = extensionValidators;
+    }
+
+    /**
      * Set the output stream to which the compiled representation will be generated.
      * @param os compiled representation output stream
      */
@@ -244,6 +256,9 @@
             if (_compileProperties != null) {
                 if (_compileProperties.get(PROCESS_CUSTOM_PROPERTIES) != null)
                     compiler.setCustomProperties((Map<QName, Node>) _compileProperties.get(PROCESS_CUSTOM_PROPERTIES));
+            }
+            if (_extensionValidators != null) {
+            	compiler.setExtensionValidators(_extensionValidators);
             }
         } catch (CompilationException ce) {
             this.invalidate();

Modified: ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/BpelCompiler.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/BpelCompiler.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/BpelCompiler.java (original)
+++ ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/BpelCompiler.java Mon Oct 15 15:48:30 2007
@@ -50,6 +50,7 @@
 import org.apache.ode.bpel.compiler.api.CompileListener;
 import org.apache.ode.bpel.compiler.api.CompilerContext;
 import org.apache.ode.bpel.compiler.api.ExpressionCompiler;
+import org.apache.ode.bpel.compiler.api.ExtensionValidator;
 import org.apache.ode.bpel.compiler.api.SourceLocation;
 import org.apache.ode.bpel.compiler.bom.Activity;
 import org.apache.ode.bpel.compiler.bom.Bpel11QNames;
@@ -178,6 +179,8 @@
 
     private URI _processURI;
 
+	private Map<QName, ExtensionValidator> _extensionValidators = new HashMap<QName, ExtensionValidator>();
+
     BpelCompiler(WSDLFactory4BPEL wsdlFactory) {
         _wsdlFactory = wsdlFactory;
         _wsdlRegistry = new WSDLRegistry(this);
@@ -1645,10 +1648,18 @@
         registerExpressionLanguage(expLangUri, (ExpressionCompiler) cls.newInstance());
     }
 
+    protected void setExtensionValidators(Map<QName, ExtensionValidator> extensionValidators) {
+    	_extensionValidators = extensionValidators;
+    }
+    
     public boolean isExtensionDeclared(String namespace) {
     	return _declaredExtensionNS.contains(namespace);
     }
 
+    public ExtensionValidator getExtensionValidator(QName extensionElementName) {
+    	return _extensionValidators.get(extensionElementName);
+    }
+    
     public List<OActivity> getActivityStack() {
         ArrayList<OActivity> rval = new ArrayList<OActivity>(_structureStack._stack);
         Collections.reverse(rval);

Modified: ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/ExtensionActivtityGenerator.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/ExtensionActivtityGenerator.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/ExtensionActivtityGenerator.java (original)
+++ ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/ExtensionActivtityGenerator.java Mon Oct 15 15:48:30 2007
@@ -19,11 +19,13 @@
 package org.apache.ode.bpel.compiler;
 
 import org.apache.ode.bpel.compiler.api.CompilationException;
+import org.apache.ode.bpel.compiler.api.ExtensionValidator;
 import org.apache.ode.bpel.compiler.bom.Activity;
 import org.apache.ode.bpel.compiler.bom.CompositeActivity;
 import org.apache.ode.bpel.compiler.bom.ExtensionActivity;
 import org.apache.ode.bpel.o.OActivity;
 import org.apache.ode.bpel.o.OExtensionActivity;
+import org.apache.ode.utils.DOMUtils;
 import org.apache.ode.utils.SerializableElement;
 import org.apache.ode.utils.msg.MessageBundle;
 import org.w3c.dom.Element;
@@ -47,7 +49,11 @@
 			if (!_context.isExtensionDeclared(child.getNamespaceURI())) {
 				throw new CompilationException(__cmsgs.errUndeclaredExtensionActivity().setSource(src));
 			}
-	        oactivity.nestedElement = new SerializableElement(child);
+	        ExtensionValidator validator = _context.getExtensionValidator(DOMUtils.getElementQName(child));
+	        if (validator != null) {
+	        	validator.validate(src);
+	        }
+			oactivity.nestedElement = new SerializableElement(child);
 	        compileChildren(oactivity, (ExtensionActivity) src);
 
         } catch (CompilationException e) {

Modified: ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/api/CompilerContext.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/api/CompilerContext.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/api/CompilerContext.java (original)
+++ ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/api/CompilerContext.java Mon Oct 15 15:48:30 2007
@@ -19,6 +19,7 @@
 package org.apache.ode.bpel.compiler.api;
 
 import java.util.List;
+import java.util.Map;
 
 import javax.wsdl.Operation;
 import javax.xml.namespace.QName;
@@ -132,4 +133,8 @@
     OActivity getCurrent();
 
     boolean isExtensionDeclared(String namespace);
+    
+    //void setExtensionValidators(Map<QName, ExtensionValidator> extensionValidators);
+    
+    ExtensionValidator getExtensionValidator(QName extensionElementName);
 }

Added: ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/api/ExtensionValidator.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/api/ExtensionValidator.java?rev=584955&view=auto
==============================================================================
--- ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/api/ExtensionValidator.java (added)
+++ ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/api/ExtensionValidator.java Mon Oct 15 15:48:30 2007
@@ -0,0 +1,33 @@
+/*
+ * 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.bpel.compiler.api;
+
+import org.apache.ode.bpel.compiler.bom.ExtensibleElement;
+
+/**
+ * Interface that allows Ode extensions to validate an extension element's
+ * content during compilation.
+ * 
+ * @author Tammo van Lessen (University of Stuttgart)
+ */
+public interface ExtensionValidator {
+
+	void validate(ExtensibleElement element) throws CompilationException;
+
+}

Added: ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensibleElement.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensibleElement.java?rev=584955&view=auto
==============================================================================
--- ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensibleElement.java (added)
+++ ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensibleElement.java Mon Oct 15 15:48:30 2007
@@ -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.bpel.compiler.bom;
+
+import org.w3c.dom.Element;
+
+/**
+ * Common interface for ExtensionActivity and AssignExtensionOperation.
+ * 
+ * @author Tammo van Lessen (University of Stuttgart)
+ */
+public interface ExtensibleElement {
+	
+	Element getNestedElement();
+	
+}

Modified: ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensionActivity.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensionActivity.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensionActivity.java (original)
+++ ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensionActivity.java Mon Oct 15 15:48:30 2007
@@ -33,7 +33,7 @@
  *
  * @author Tammo van Lessen (University of Stuttgart)
  */
-public class ExtensionActivity extends CompositeActivity {
+public class ExtensionActivity extends CompositeActivity implements ExtensibleElement {
 	private Activity _childActivity;
 
 	public ExtensionActivity(Element el) {
@@ -92,6 +92,10 @@
 		}
 
 		return _childActivity.getChildren(Activity.class);
+	}
+
+	public Element getNestedElement() {
+		return getFirstExtensibilityElement();
 	}
 
 }

Modified: ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensionAssignOperation.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensionAssignOperation.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensionAssignOperation.java (original)
+++ ode/trunk/bpel-compiler/src/main/java/org/apache/ode/bpel/compiler/bom/ExtensionAssignOperation.java Mon Oct 15 15:48:30 2007
@@ -28,7 +28,8 @@
  *  
  * @author Tammo van Lessen (University of Stuttgart)
  */
-public class ExtensionAssignOperation extends BpelObject implements AssignOperation {
+public class ExtensionAssignOperation extends BpelObject implements AssignOperation,
+																	ExtensibleElement {
 	private Element _childElement;
 	
     public ExtensionAssignOperation(Element el) {
@@ -36,7 +37,9 @@
     }
 
     public Element getNestedElement() {
-        if (_childElement == null) {
+        //XXX
+    	//return getFirstExtensibilityElement(); 
+    	if (_childElement == null) {
 	    	NodeList nl = getElement().getChildNodes();
 	        for (int i = 0; i < nl.getLength(); ++i) {
 	            Node node = nl.item(i);

Modified: ode/trunk/bpel-compiler/src/test/java/org/apache/ode/bpel/compiler/XPathTest.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-compiler/src/test/java/org/apache/ode/bpel/compiler/XPathTest.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-compiler/src/test/java/org/apache/ode/bpel/compiler/XPathTest.java (original)
+++ ode/trunk/bpel-compiler/src/test/java/org/apache/ode/bpel/compiler/XPathTest.java Mon Oct 15 15:48:30 2007
@@ -32,6 +32,7 @@
 import org.apache.ode.bpel.compiler.api.CompilationException;
 import org.apache.ode.bpel.compiler.api.CompilerContext;
 import org.apache.ode.bpel.compiler.api.ExpressionCompiler;
+import org.apache.ode.bpel.compiler.api.ExtensionValidator;
 import org.apache.ode.bpel.compiler.api.SourceLocation;
 import org.apache.ode.bpel.compiler.bom.Activity;
 import org.apache.ode.bpel.compiler.bom.BpelObject;
@@ -270,5 +271,9 @@
 
 	public boolean isExtensionDeclared(String namespace) {
 		return false;
+	}
+
+	public ExtensionValidator getExtensionValidator(QName extensionElementName) {
+		return null;
 	}
 }

Modified: ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java (original)
+++ ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelRuntimeContextImpl.java Mon Oct 15 15:48:30 2007
@@ -42,10 +42,12 @@
 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.eapi.AbstractExtensionBundle;
-import org.apache.ode.bpel.eapi.ExtensionContext;
-import org.apache.ode.bpel.eapi.ExtensionOperation;
-import org.apache.ode.bpel.evt.*;
+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.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;
@@ -76,6 +78,9 @@
 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.runtime.extension.AbstractExtensionBundle;
+import org.apache.ode.bpel.runtime.extension.ExtensionContext;
+import org.apache.ode.bpel.runtime.extension.ExtensionOperation;
 import org.apache.ode.jacob.JacobRunnable;
 import org.apache.ode.jacob.vpu.ExecutionQueueImpl;
 import org.apache.ode.jacob.vpu.JacobVPU;
@@ -83,10 +88,9 @@
 import org.apache.ode.utils.GUID;
 import org.apache.ode.utils.Namespaces;
 import org.apache.ode.utils.ObjectPrinter;
-import org.apache.ode.utils.SerializableElement;
+import org.w3c.dom.Document;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
-import org.w3c.dom.Document;
 
 /**
  * 
@@ -1190,7 +1194,7 @@
         _forceFlush = true;
     }
     
-	public void executeExtension(QName extensionId, ExtensionContext context, SerializableElement element, ExtensionResponseChannel extResponseChannel) throws FaultException {
+	public void executeExtension(QName extensionId, ExtensionContext context, Element element, ExtensionResponseChannel extResponseChannel) throws FaultException {
 		__log.debug("Execute extension activity");
 		final String channelId = extResponseChannel.export();
 		ExtensionOperation ea = createExtensionActivityImplementation(extensionId);
@@ -1207,7 +1211,6 @@
 		}
 		
 		try {
-			// should be running in a pooled thread
 			ea.run(context, element);
 			completeExtensionExecution(channelId, null);
 		} catch (RuntimeException e) {

Modified: ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelServerImpl.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelServerImpl.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelServerImpl.java (original)
+++ ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/BpelServerImpl.java Mon Oct 15 15:48:30 2007
@@ -42,7 +42,6 @@
 import org.apache.ode.bpel.dao.BpelDAOConnectionFactory;
 import org.apache.ode.bpel.dao.MessageExchangeDAO;
 import org.apache.ode.bpel.dao.ProcessDAO;
-import org.apache.ode.bpel.eapi.AbstractExtensionBundle;
 import org.apache.ode.bpel.evt.BpelEvent;
 import org.apache.ode.bpel.iapi.BindingContext;
 import org.apache.ode.bpel.iapi.BpelEngineException;
@@ -62,6 +61,7 @@
 import org.apache.ode.bpel.iapi.Scheduler.JobProcessorException;
 import org.apache.ode.bpel.intercept.MessageExchangeInterceptor;
 import org.apache.ode.bpel.o.OProcess;
+import org.apache.ode.bpel.runtime.extension.AbstractExtensionBundle;
 import org.apache.ode.utils.msg.MessageBundle;
 import org.apache.ode.utils.stl.CollectionsX;
 import org.apache.ode.utils.stl.MemberOfFunction;

Modified: ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/Contexts.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/Contexts.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/Contexts.java (original)
+++ ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/engine/Contexts.java Mon Oct 15 15:48:30 2007
@@ -33,7 +33,6 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.dao.BpelDAOConnectionFactory;
-import org.apache.ode.bpel.eapi.AbstractExtensionBundle;
 import org.apache.ode.bpel.iapi.BindingContext;
 import org.apache.ode.bpel.iapi.BpelEngineException;
 import org.apache.ode.bpel.iapi.BpelEventListener;
@@ -41,6 +40,7 @@
 import org.apache.ode.bpel.iapi.MessageExchangeContext;
 import org.apache.ode.bpel.iapi.Scheduler;
 import org.apache.ode.bpel.intercept.MessageExchangeInterceptor;
+import org.apache.ode.bpel.runtime.extension.AbstractExtensionBundle;
 
 /**
  * Aggregation of all the contexts provided to the BPEL engine by the integration layer.

Modified: ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ASSIGN.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ASSIGN.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ASSIGN.java (original)
+++ ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ASSIGN.java Mon Oct 15 15:48:30 2007
@@ -28,7 +28,6 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.common.FaultException;
 import org.apache.ode.bpel.compiler.bom.Bpel20QNames;
-import org.apache.ode.bpel.eapi.ExtensionContext;
 import org.apache.ode.bpel.evt.PartnerLinkModificationEvent;
 import org.apache.ode.bpel.evt.ScopeEvent;
 import org.apache.ode.bpel.evt.VariableModificationEvent;
@@ -49,6 +48,7 @@
 import org.apache.ode.bpel.runtime.channels.ExtensionResponseChannel;
 import org.apache.ode.bpel.runtime.channels.ExtensionResponseChannelListener;
 import org.apache.ode.bpel.runtime.channels.FaultData;
+import org.apache.ode.bpel.runtime.extension.ExtensionContext;
 import org.apache.ode.utils.DOMUtils;
 import org.apache.ode.utils.Namespaces;
 import org.apache.ode.utils.msg.MessageBundle;
@@ -587,7 +587,7 @@
 	    	final ExtensionContext helper = new ExtensionContextImpl(_self.o, _scopeFrame, getBpelRuntimeContext());
 	    	final ExtensionResponseChannel responseChannel = newChannel(ExtensionResponseChannel.class);
 
-    		getBpelRuntimeContext().executeExtension(DOMUtils.getElementQName(eao.nestedElement.getElement()), helper, eao.nestedElement, responseChannel);
+    		getBpelRuntimeContext().executeExtension(DOMUtils.getElementQName(eao.nestedElement.getElement()), helper, eao.nestedElement.getElement(), responseChannel);
 
     		object(new ExtensionResponseChannelListener(responseChannel) {
 				private static final long serialVersionUID = 1L;

Modified: ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelRuntimeContext.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelRuntimeContext.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelRuntimeContext.java (original)
+++ ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/BpelRuntimeContext.java Mon Oct 15 15:48:30 2007
@@ -20,7 +20,6 @@
 
 import org.apache.ode.bpel.common.CorrelationKey;
 import org.apache.ode.bpel.common.FaultException;
-import org.apache.ode.bpel.eapi.ExtensionContext;
 import org.apache.ode.bpel.evt.ProcessInstanceEvent;
 import org.apache.ode.bpel.o.OMessageVarType;
 import org.apache.ode.bpel.o.OMessageVarType.Part;
@@ -33,6 +32,7 @@
 import org.apache.ode.bpel.runtime.channels.PickResponseChannel;
 import org.apache.ode.bpel.runtime.channels.TimerResponseChannel;
 import org.apache.ode.bpel.runtime.channels.ActivityRecoveryChannel;
+import org.apache.ode.bpel.runtime.extension.ExtensionContext;
 import org.apache.ode.utils.SerializableElement;
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
@@ -260,5 +260,5 @@
 
     void releasePartnerMex(String mexId);
     
-    void executeExtension(QName extensionId, ExtensionContext context, SerializableElement element, ExtensionResponseChannel extResponseChannel) throws FaultException;
+    void executeExtension(QName extensionId, ExtensionContext context, Element element, ExtensionResponseChannel extResponseChannel) throws FaultException;
 }

Modified: ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EXTENSIONACTIVITY.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EXTENSIONACTIVITY.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EXTENSIONACTIVITY.java (original)
+++ ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/EXTENSIONACTIVITY.java Mon Oct 15 15:48:30 2007
@@ -27,11 +27,11 @@
 import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.common.FaultException;
 import org.apache.ode.bpel.compiler.bom.Bpel20QNames;
-import org.apache.ode.bpel.eapi.ExtensionContext;
 import org.apache.ode.bpel.o.OExtensionActivity;
 import org.apache.ode.bpel.runtime.channels.ExtensionResponseChannel;
 import org.apache.ode.bpel.runtime.channels.ExtensionResponseChannelListener;
 import org.apache.ode.bpel.runtime.channels.FaultData;
+import org.apache.ode.bpel.runtime.extension.ExtensionContext;
 import org.apache.ode.utils.DOMUtils;
 import org.apache.ode.utils.SerializableElement;
 
@@ -58,7 +58,7 @@
         	final ExtensionResponseChannel responseChannel = newChannel(ExtensionResponseChannel.class);
         	final ExtensionContext helper = new ExtensionContextImpl(_self.o, _scopeFrame, getBpelRuntimeContext());
         	
-        	getBpelRuntimeContext().executeExtension(DOMUtils.getElementQName(_oext.nestedElement.getElement()), helper, new SerializableElement(_oext.nestedElement.getElement()), responseChannel);
+        	getBpelRuntimeContext().executeExtension(DOMUtils.getElementQName(_oext.nestedElement.getElement()), helper, _oext.nestedElement.getElement(), responseChannel);
         	
             object(new ExtensionResponseChannelListener(responseChannel) {
 				private static final long serialVersionUID = -1L;

Modified: ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ExtensionContextImpl.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ExtensionContextImpl.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ExtensionContextImpl.java (original)
+++ ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/ExtensionContextImpl.java Mon Oct 15 15:48:30 2007
@@ -19,16 +19,15 @@
 package org.apache.ode.bpel.runtime;
 
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import org.apache.ode.bpel.common.FaultException;
-import org.apache.ode.bpel.eapi.ExtensionContext;
 import org.apache.ode.bpel.o.OActivity;
 import org.apache.ode.bpel.o.OLink;
 import org.apache.ode.bpel.o.OScope;
 import org.apache.ode.bpel.o.OProcess.OProperty;
 import org.apache.ode.bpel.o.OScope.Variable;
+import org.apache.ode.bpel.runtime.extension.ExtensionContext;
 import org.w3c.dom.Node;
 
 
@@ -113,5 +112,9 @@
 
 	public OActivity getOActivity() {
 		return _activity;
+	}
+
+	public BpelRuntimeContext getBpelRuntimeContext() {
+		return _context;
 	}
 }

Copied: ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/AbstractExtensionBundle.java (from r583317, ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/AbstractExtensionBundle.java)
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/AbstractExtensionBundle.java?p2=ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/AbstractExtensionBundle.java&p1=ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/AbstractExtensionBundle.java&r1=583317&r2=584955&rev=584955&view=diff
==============================================================================
--- ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/AbstractExtensionBundle.java (original)
+++ ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/AbstractExtensionBundle.java Mon Oct 15 15:48:30 2007
@@ -16,13 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.ode.bpel.eapi;
+package org.apache.ode.bpel.runtime.extension;
 
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
+import javax.xml.namespace.QName;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.ode.bpel.compiler.api.ExtensionValidator;
+
 /**
 * Abstract class that bundles and registers <code>&lt;extensionActivity&gt;</code> and
 * <code>&lt;extensionAssignOperation&gt;</code> implementations related to a particular namespace.
@@ -30,6 +36,7 @@
 * @author Tammo van Lessen (University of Stuttgart)
 */
 public abstract class AbstractExtensionBundle {
+	private static Log __log = LogFactory.getLog(AbstractExtensionBundle.class);
 	private Map<String, Class<? extends ExtensionOperation>> extensionsByName = new HashMap<String, Class<? extends ExtensionOperation>>();
 
 	/**
@@ -68,4 +75,18 @@
 		return getExtensionOperationClass(localName).newInstance();
 	}
 
+	public final Map<QName, ExtensionValidator> getExtensionValidators() {
+		Map<QName, ExtensionValidator> result = new HashMap<QName, ExtensionValidator>();
+		String ns = getNamespaceURI();
+		for (String localName : extensionsByName.keySet()) {
+			if (ExtensionValidator.class.isAssignableFrom(extensionsByName.get(localName))) {
+				try {
+					result.put(new QName(ns, localName), (ExtensionValidator)getExtensionOperationInstance(localName));
+				} catch (Exception e) {
+					__log.warn("Could not instantiate extension validator for '{" + ns + "}" + localName);
+				}
+			}
+		}
+		return result;
+	}
 }

Added: ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/AbstractExtensionOperation.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/AbstractExtensionOperation.java?rev=584955&view=auto
==============================================================================
--- ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/AbstractExtensionOperation.java (added)
+++ ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/AbstractExtensionOperation.java Mon Oct 15 15:48:30 2007
@@ -0,0 +1,37 @@
+/*
+ * 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.bpel.runtime.extension;
+
+import org.apache.ode.bpel.compiler.api.CompilationException;
+import org.apache.ode.bpel.compiler.api.ExtensionValidator;
+import org.apache.ode.bpel.compiler.bom.ExtensibleElement;
+
+/**
+ * Base class for creating new extension implementations.
+ * 
+ * @author Tammo van Lessen (University of Stuttgart)
+ */
+public abstract class AbstractExtensionOperation implements ExtensionValidator,
+															ExtensionOperation {
+
+	public void validate(ExtensibleElement element) throws CompilationException {
+		// default behavior: pass
+	}
+
+}

Copied: ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/ExtensionContext.java (from r583317, ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/ExtensionContext.java)
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/ExtensionContext.java?p2=ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/ExtensionContext.java&p1=ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/ExtensionContext.java&r1=583317&r2=584955&rev=584955&view=diff
==============================================================================
--- ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/ExtensionContext.java (original)
+++ ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/ExtensionContext.java Mon Oct 15 15:48:30 2007
@@ -16,9 +16,8 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.ode.bpel.eapi;
+package org.apache.ode.bpel.runtime.extension;
 
-import java.util.List;
 import java.util.Map;
 
 import org.apache.ode.bpel.common.FaultException;
@@ -26,6 +25,7 @@
 import org.apache.ode.bpel.o.OLink;
 import org.apache.ode.bpel.o.OProcess;
 import org.apache.ode.bpel.o.OScope;
+import org.apache.ode.bpel.runtime.BpelRuntimeContext;
 import org.w3c.dom.Node;
 
 
@@ -128,5 +128,9 @@
      * Low-level-method
      */
     OActivity getOActivity();
+    
+    BpelRuntimeContext getBpelRuntimeContext();
+    
+    //ScopeFrame getScopeFrame();
     
 }

Copied: ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/ExtensionOperation.java (from r583317, ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/ExtensionOperation.java)
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/ExtensionOperation.java?p2=ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/ExtensionOperation.java&p1=ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/ExtensionOperation.java&r1=583317&r2=584955&rev=584955&view=diff
==============================================================================
--- ode/trunk/bpel-api/src/main/java/org/apache/ode/bpel/eapi/ExtensionOperation.java (original)
+++ ode/trunk/bpel-runtime/src/main/java/org/apache/ode/bpel/runtime/extension/ExtensionOperation.java Mon Oct 15 15:48:30 2007
@@ -16,10 +16,10 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.ode.bpel.eapi;
+package org.apache.ode.bpel.runtime.extension;
 
 import org.apache.ode.bpel.common.FaultException;
-import org.apache.ode.utils.SerializableElement;
+import org.w3c.dom.Element;
 
 /**
  * This is the basis interface for implementations of  
@@ -35,5 +35,5 @@
  */
 public interface ExtensionOperation {
 
-	void run(ExtensionContext context, SerializableElement element) throws FaultException;
+	void run(ExtensionContext context, Element element) throws FaultException;
 }

Modified: ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/elang/xpath20/runtime/MockCompilerContext.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/elang/xpath20/runtime/MockCompilerContext.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/elang/xpath20/runtime/MockCompilerContext.java (original)
+++ ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/elang/xpath20/runtime/MockCompilerContext.java Mon Oct 15 15:48:30 2007
@@ -28,6 +28,7 @@
 
 import org.apache.ode.bpel.compiler.api.CompilationException;
 import org.apache.ode.bpel.compiler.api.CompilerContext;
+import org.apache.ode.bpel.compiler.api.ExtensionValidator;
 import org.apache.ode.bpel.compiler.api.SourceLocation;
 import org.apache.ode.bpel.compiler.bom.Activity;
 import org.apache.ode.bpel.compiler.bom.BpelObject;
@@ -206,7 +207,9 @@
     }
     
 	public boolean isExtensionDeclared(String namespace) {
-		// TODO Auto-generated method stub
 		return false;
+	}
+	public ExtensionValidator getExtensionValidator(QName extensionElementName) {
+		return null;
 	}
 }

Modified: ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/runtime/CoreBpelTest.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/runtime/CoreBpelTest.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/runtime/CoreBpelTest.java (original)
+++ ode/trunk/bpel-runtime/src/test/java/org/apache/ode/bpel/runtime/CoreBpelTest.java Mon Oct 15 15:48:30 2007
@@ -18,41 +18,42 @@
  */
 package org.apache.ode.bpel.runtime;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.util.Collection;
+import java.util.Date;
+
+import javax.wsdl.Operation;
+import javax.xml.namespace.QName;
+
 import junit.framework.TestCase;
+
 import org.apache.ode.bpel.common.CorrelationKey;
 import org.apache.ode.bpel.common.FaultException;
-import org.apache.ode.bpel.eapi.ExtensionContext;
 import org.apache.ode.bpel.evt.ProcessInstanceEvent;
 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.o.OMessageVarType.Part;
+import org.apache.ode.bpel.runtime.channels.ActivityRecoveryChannel;
 import org.apache.ode.bpel.runtime.channels.ExtensionResponseChannel;
 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.bpel.runtime.channels.ActivityRecoveryChannel;
+import org.apache.ode.bpel.runtime.extension.ExtensionContext;
 import org.apache.ode.jacob.vpu.ExecutionQueueImpl;
 import org.apache.ode.jacob.vpu.JacobVPU;
-import org.apache.ode.utils.SerializableElement;
 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.Collection;
-import java.util.Date;
-
 /**
  * Test core BPEL processing capabilities.
  */
@@ -418,7 +419,7 @@
     }
 
 	public void executeExtension(QName extensionId, ExtensionContext context,
-			SerializableElement element,
+			Element element,
 			ExtensionResponseChannel extResponseChannel) throws FaultException {
 		// TODO Auto-generated method stub
 		

Modified: ode/trunk/bpel-store/src/main/java/org/apache/ode/store/DeploymentUnitDir.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-store/src/main/java/org/apache/ode/store/DeploymentUnitDir.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-store/src/main/java/org/apache/ode/store/DeploymentUnitDir.java (original)
+++ ode/trunk/bpel-store/src/main/java/org/apache/ode/store/DeploymentUnitDir.java Mon Oct 15 15:48:30 2007
@@ -23,6 +23,7 @@
 import org.apache.ode.bpel.compiler.BpelC;
 import org.apache.ode.bpel.compiler.DefaultResourceFinder;
 import org.apache.ode.bpel.compiler.WSDLLocatorImpl;
+import org.apache.ode.bpel.compiler.api.ExtensionValidator;
 import org.apache.ode.bpel.compiler.wsdl.Definition4BPEL;
 import org.apache.ode.bpel.compiler.wsdl.WSDLFactory4BPEL;
 import org.apache.ode.bpel.compiler.wsdl.WSDLFactoryBPEL20;
@@ -67,7 +68,8 @@
 
     private HashMap<QName, CBPInfo> _processes = new HashMap<QName,CBPInfo>();
     private HashMap<QName, TDeployment.Process> _processInfo = new HashMap<QName,TDeployment.Process>();
-
+    private Map<QName, ExtensionValidator> _extensionValidators;
+    
     private volatile DeployDocument _dd;
     private volatile DocumentRegistry _docRegistry;
 
@@ -156,6 +158,7 @@
             bpelc.setProcessWSDL(bpel11wsdl.toURI());
         
         bpelc.setCompileProperties(prepareCompileProperties(bpelFile));
+        bpelc.setExtensionValidators(_extensionValidators);
         bpelc.setBaseDirectory(_duDirectory);
         try {
             bpelc.compile(bpelFile);
@@ -364,4 +367,7 @@
         _version = version;
     }
 
+    public void setExtensionValidators(Map<QName, ExtensionValidator> extensionValidators) {
+    	_extensionValidators = extensionValidators;
+    }
 }

Modified: ode/trunk/bpel-store/src/main/java/org/apache/ode/store/ProcessStoreImpl.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-store/src/main/java/org/apache/ode/store/ProcessStoreImpl.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-store/src/main/java/org/apache/ode/store/ProcessStoreImpl.java (original)
+++ ode/trunk/bpel-store/src/main/java/org/apache/ode/store/ProcessStoreImpl.java Mon Oct 15 15:48:30 2007
@@ -18,9 +18,30 @@
  */
 package org.apache.ode.store;
 
+import java.io.File;
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import javax.sql.DataSource;
+import javax.xml.namespace.QName;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.ode.bpel.compiler.api.CompilationException;
+import org.apache.ode.bpel.compiler.api.ExtensionValidator;
 import org.apache.ode.bpel.dd.DeployDocument;
 import org.apache.ode.bpel.dd.TDeployment;
 import org.apache.ode.bpel.iapi.ContextException;
@@ -38,25 +59,6 @@
 import org.w3c.dom.Element;
 import org.w3c.dom.Node;
 
-import javax.sql.DataSource;
-import javax.xml.namespace.QName;
-import java.io.File;
-import java.io.IOException;
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 /**
  * <p>
  * JDBC-based implementation of a process store. Also provides an "in-memory" store by way of HSQL database.
@@ -85,6 +87,8 @@
     private Map<QName, ProcessConfImpl> _processes = new HashMap<QName, ProcessConfImpl>();
 
     private Map<String, DeploymentUnitDir> _deploymentUnits = new HashMap<String, DeploymentUnitDir>();
+    
+    private Map<QName, ExtensionValidator> _extensionValidators = new HashMap<QName, ExtensionValidator>();
 
     /** Guards access to the _processes and _deploymentUnits */
     private final ReadWriteLock _rw = new ReentrantReadWriteLock();
@@ -161,6 +165,7 @@
 
         // Create the DU and compile/scan it before acquiring lock.
         final DeploymentUnitDir du = new DeploymentUnitDir(deploymentUnitDirectory);
+        du.setExtensionValidators(_extensionValidators);
         try {
             du.compile();
         } catch (CompilationException ce) {
@@ -770,4 +775,8 @@
         }
         return old;
     }
+
+	public void setExtensionValidators(Map<QName, ExtensionValidator> extensionValidators) {
+		_extensionValidators = extensionValidators;
+	}
 }

Modified: ode/trunk/bpel-test/src/main/java/org/apache/ode/test/BPELTestAbstract.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-test/src/main/java/org/apache/ode/test/BPELTestAbstract.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-test/src/main/java/org/apache/ode/test/BPELTestAbstract.java (original)
+++ ode/trunk/bpel-test/src/main/java/org/apache/ode/test/BPELTestAbstract.java Mon Oct 15 15:48:30 2007
@@ -38,7 +38,6 @@
 
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.ode.bpel.dao.BpelDAOConnectionFactory;
-import org.apache.ode.bpel.eapi.AbstractExtensionBundle;
 import org.apache.ode.bpel.engine.BpelServerImpl;
 import org.apache.ode.bpel.iapi.InvocationStyle;
 import org.apache.ode.bpel.iapi.Message;
@@ -51,6 +50,7 @@
 import org.apache.ode.bpel.iapi.MessageExchange.Status;
 import org.apache.ode.bpel.iapi.MyRoleMessageExchange.CorrelationStatus;
 import org.apache.ode.bpel.memdao.BpelDAOConnectionFactoryImpl;
+import org.apache.ode.bpel.runtime.extension.AbstractExtensionBundle;
 import org.apache.ode.dao.jpa.BPELDAOConnectionFactoryImpl;
 import org.apache.ode.il.MockScheduler;
 import org.apache.ode.store.ProcessConfImpl;

Added: ode/trunk/bpel-test/src/main/java/org/apache/ode/test/MockExtensionContext.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-test/src/main/java/org/apache/ode/test/MockExtensionContext.java?rev=584955&view=auto
==============================================================================
--- ode/trunk/bpel-test/src/main/java/org/apache/ode/test/MockExtensionContext.java (added)
+++ ode/trunk/bpel-test/src/main/java/org/apache/ode/test/MockExtensionContext.java Mon Oct 15 15:48:30 2007
@@ -0,0 +1,94 @@
+/*
+ * 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 java.util.HashMap;
+import java.util.Map;
+
+import org.apache.ode.bpel.common.FaultException;
+import org.apache.ode.bpel.o.OActivity;
+import org.apache.ode.bpel.o.OLink;
+import org.apache.ode.bpel.o.OProcess.OProperty;
+import org.apache.ode.bpel.o.OScope.Variable;
+import org.apache.ode.bpel.runtime.BpelRuntimeContext;
+import org.apache.ode.bpel.runtime.extension.ExtensionContext;
+import org.apache.ode.utils.DOMUtils;
+import org.w3c.dom.Node;
+
+/**
+ * Very simple mock implementation of the ExtensionContext interface.
+ * 
+ * @author Tammo van Lessen (University of Stuttgart)
+ */
+public class MockExtensionContext implements ExtensionContext {
+	private Map<String, Node> variables = new HashMap<String, Node>();
+	
+	public Map<String, Node> getVariables() {
+		return variables;
+	}
+	
+	public Long getProcessId() {
+		return 0L;
+	}
+
+	public Node readVariable(String variableName) throws FaultException {
+		System.out.println("Reading " + variableName);
+		return variables.get(variableName);
+	}
+
+	public void writeVariable(String variableName, Node value)
+			throws FaultException {
+		variables.put(variableName, value);
+		System.out.println("Storing in " + variableName + ": " + DOMUtils.domToString(value));
+	}
+
+	public String getActivityName() {
+		return "mockActivity";
+	}
+	
+	public OActivity getOActivity() {
+		throw new UnsupportedOperationException("This method is not available in this mock implementation.");
+	}
+	
+	public BpelRuntimeContext getBpelRuntimeContext() {
+		throw new UnsupportedOperationException("This method is not available in this mock implementation.");
+	}
+	
+	public Map<String, Variable> getVisibleVariables()
+		throws FaultException {
+		throw new UnsupportedOperationException("This method is not available in this mock implementation.");
+	}
+	
+	public boolean isLinkActive(OLink olink) throws FaultException {
+		throw new UnsupportedOperationException("This method is not available in this mock implementation.");
+	}
+	
+	public String readMessageProperty(Variable variable, OProperty property)
+		throws FaultException {
+		throw new UnsupportedOperationException("This method is not available in this mock implementation.");
+	}
+	
+	public Node readVariable(Variable variable) throws FaultException {
+		throw new UnsupportedOperationException("This method is not available in this mock implementation.");
+	}
+	
+	public void writeVariable(Variable variable, Node value) throws FaultException {
+		throw new UnsupportedOperationException("This method is not available in this mock implementation.");
+	}
+}
\ No newline at end of file

Modified: ode/trunk/bpel-test/src/test/java/org/apache/ode/test/ExtensibilityTest.java
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-test/src/test/java/org/apache/ode/test/ExtensibilityTest.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-test/src/test/java/org/apache/ode/test/ExtensibilityTest.java (original)
+++ ode/trunk/bpel-test/src/test/java/org/apache/ode/test/ExtensibilityTest.java Mon Oct 15 15:48:30 2007
@@ -18,16 +18,22 @@
  */
 package org.apache.ode.test;
 
+import javax.xml.namespace.QName;
+
 import org.apache.ode.bpel.common.FaultException;
-import org.apache.ode.bpel.eapi.AbstractExtensionBundle;
-import org.apache.ode.bpel.eapi.ExtensionContext;
-import org.apache.ode.bpel.eapi.ExtensionOperation;
+import org.apache.ode.bpel.compiler.api.CompilationException;
+import org.apache.ode.bpel.compiler.api.CompilationMessage;
+import org.apache.ode.bpel.compiler.bom.ExtensibleElement;
 import org.apache.ode.bpel.iapi.BpelEngineException;
+import org.apache.ode.bpel.runtime.extension.AbstractExtensionBundle;
+import org.apache.ode.bpel.runtime.extension.AbstractExtensionOperation;
+import org.apache.ode.bpel.runtime.extension.ExtensionContext;
+import org.apache.ode.bpel.runtime.extension.ExtensionOperation;
 import org.apache.ode.utils.DOMUtils;
-import org.apache.ode.utils.SerializableElement;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.w3c.dom.Element;
 
 /**
  * Test ODE's extensibility
@@ -79,8 +85,20 @@
 		teb.recycle();
 	}
 
+	@Test public void testExtensionActivityCompilerError() throws Throwable {
+		_server.registerExtensionBundle(teb);
+		TestExtensionBundle.cmpString = "error";
+		go("/bpel/2.0/TestExtensionActivity");
+        Deployment deployment = new Deployment(makeDeployDir("/bpel/2.0/TestExtensionActivityMustUnderstand"));
+        deployment.expectedException = CompilationException.class;
+        doDeployment(deployment);
+		_server.unregisterExtensionBundle(teb.getNamespaceURI());
+		teb.recycle();
+	}
+	
 	private static class TestExtensionBundle extends AbstractExtensionBundle {
 		private static boolean wasExecuted = false;
+		private static String cmpString = "test";
 		
 		public String getNamespaceURI() {
 			return "urn:ode:test-extension-bundle";
@@ -88,6 +106,7 @@
 
 		public void registerExtensionActivities() {
 			registerExtensionOperation("doIt", TestExtensionActivity.class);
+			registerExtensionOperation("doIt2", TestExtensionValidatorActivity.class);
 			registerExtensionOperation("doAssign", TestExtensionAssignOperation.class);
 		}
 		
@@ -97,6 +116,7 @@
 		
 		public void recycle() {
 			wasExecuted = false;
+			cmpString = "test";
 		}
 	}
 
@@ -104,7 +124,7 @@
 		private static final long serialVersionUID = 1L;
 
 		public void run(ExtensionContext context,
-				SerializableElement element) throws FaultException {
+				Element element) throws FaultException {
 			TestExtensionBundle.wasExecuted = true;
 		}
 	}
@@ -112,7 +132,7 @@
 	public static class TestExtensionAssignOperation implements ExtensionOperation {
 		private static final long serialVersionUID = 1L;
 
-		public void run(ExtensionContext context, SerializableElement element)
+		public void run(ExtensionContext context, Element element)
 				throws FaultException {
 			//Node val = context.readVariable("myVar");
 			StringBuffer sb = new StringBuffer("<?xml version=\"1.0\" encoding=\"UTF-8\"?>\n");
@@ -122,6 +142,23 @@
 			} catch (Exception e) {
 				e.printStackTrace();
 				Assert.fail();
+			}
+		}
+	}
+
+	public static class TestExtensionValidatorActivity extends AbstractExtensionOperation {
+		private static final long serialVersionUID = 1L;
+
+		public void run(ExtensionContext context,
+				Element element) throws FaultException {
+			TestExtensionBundle.wasExecuted = true;
+		}
+
+		@Override
+		public void validate(ExtensibleElement element)
+				throws CompilationException {
+			if (element.getNestedElement().getTextContent().trim().equals(TestExtensionBundle.cmpString)) {
+				throw new CompilationException(new CompilationMessage());
 			}
 		}
 	}

Modified: ode/trunk/bpel-test/src/test/resources/bpel/2.0/TestExtensionActivity/ExtensionActivity.bpel
URL: http://svn.apache.org/viewvc/ode/trunk/bpel-test/src/test/resources/bpel/2.0/TestExtensionActivity/ExtensionActivity.bpel?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/bpel-test/src/test/resources/bpel/2.0/TestExtensionActivity/ExtensionActivity.bpel (original)
+++ ode/trunk/bpel-test/src/test/resources/bpel/2.0/TestExtensionActivity/ExtensionActivity.bpel Mon Oct 15 15:48:30 2007
@@ -68,6 +68,10 @@
       	<ext:doIt/>
       </extensionActivity>
 
+      <extensionActivity>
+      	<ext:doIt2>test</ext:doIt2>
+      </extensionActivity>
+
 	  </flow>
        <reply name="end"  
               partnerLink="helloPartnerLink"

Modified: ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/ExtensionContextWrapper.java
URL: http://svn.apache.org/viewvc/ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/ExtensionContextWrapper.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/ExtensionContextWrapper.java (original)
+++ ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/ExtensionContextWrapper.java Mon Oct 15 15:48:30 2007
@@ -30,7 +30,7 @@
 import javax.xml.transform.stream.StreamResult;
 
 import org.apache.ode.bpel.common.FaultException;
-import org.apache.ode.bpel.eapi.ExtensionContext;
+import org.apache.ode.bpel.runtime.extension.ExtensionContext;
 import org.mozilla.javascript.Context;
 import org.mozilla.javascript.Function;
 import org.mozilla.javascript.Scriptable;

Modified: ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/JSExtensionBundle.java
URL: http://svn.apache.org/viewvc/ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/JSExtensionBundle.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/JSExtensionBundle.java (original)
+++ ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/JSExtensionBundle.java Mon Oct 15 15:48:30 2007
@@ -18,7 +18,7 @@
  */
 package org.apache.ode.extension.e4x;
 
-import org.apache.ode.bpel.eapi.AbstractExtensionBundle;
+import org.apache.ode.bpel.runtime.extension.AbstractExtensionBundle;
 
 /**
  * Implementation of a Javascript extension bundle.

Modified: ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/JSExtensionOperation.java
URL: http://svn.apache.org/viewvc/ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/JSExtensionOperation.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/JSExtensionOperation.java (original)
+++ ode/trunk/extensions/e4x/src/main/java/org/apache/ode/extension/e4x/JSExtensionOperation.java Mon Oct 15 15:48:30 2007
@@ -21,24 +21,24 @@
 import javax.xml.namespace.QName;
 
 import org.apache.ode.bpel.common.FaultException;
-import org.apache.ode.bpel.eapi.ExtensionContext;
-import org.apache.ode.bpel.eapi.ExtensionOperation;
-import org.apache.ode.utils.SerializableElement;
+import org.apache.ode.bpel.runtime.extension.AbstractExtensionOperation;
+import org.apache.ode.bpel.runtime.extension.ExtensionContext;
 import org.mozilla.javascript.Context;
 import org.mozilla.javascript.ContextFactory;
 import org.mozilla.javascript.Scriptable;
 import org.mozilla.javascript.ScriptableObject;
 import org.mozilla.javascript.xml.XMLLib;
 import org.mozilla.javascript.xml.XMLLib.Factory;
+import org.w3c.dom.Element;
 
 /**
  * Implementation of a Javascript extension assign operation.
  * 
  * @author Tammo van Lessen (University of Stuttgart)
  */
-public class JSExtensionOperation implements ExtensionOperation {
+public class JSExtensionOperation extends AbstractExtensionOperation {
 	
-	public void run(ExtensionContext context, SerializableElement element) throws FaultException {
+	public void run(ExtensionContext context, Element element) throws FaultException {
 
 		ContextFactory contextFactory = new ContextFactory() {
 			//Enforce usage of plain DOM
@@ -54,7 +54,7 @@
 			ScriptableObject.defineClass(scope, ExtensionContextWrapper.class);
 			Scriptable wrappedContext = ctx.newObject(scope, "ExtensionContext", new Object[] {context, ctx});
 			ScriptableObject.putProperty(scope, "context", wrappedContext);
-			String source = element.getElement().getTextContent();
+			String source = element.getTextContent();
 			ctx.evaluateString(scope, source, context.getActivityName(), 1, null);
 		} catch (Exception e) {
 			throw new FaultException(new QName("ExtensionEvaluationFault", JSExtensionBundle.NS), e.getMessage());

Modified: ode/trunk/extensions/e4x/src/test/java/org/apache/ode/extension/e4x/JSOperationTest.java
URL: http://svn.apache.org/viewvc/ode/trunk/extensions/e4x/src/test/java/org/apache/ode/extension/e4x/JSOperationTest.java?rev=584955&r1=584954&r2=584955&view=diff
==============================================================================
--- ode/trunk/extensions/e4x/src/test/java/org/apache/ode/extension/e4x/JSOperationTest.java (original)
+++ ode/trunk/extensions/e4x/src/test/java/org/apache/ode/extension/e4x/JSOperationTest.java Mon Oct 15 15:48:30 2007
@@ -18,92 +18,20 @@
  */
 package org.apache.ode.extension.e4x;
 
-import java.util.HashMap;
-import java.util.Map;
 
-import org.apache.ode.bpel.common.FaultException;
-import org.apache.ode.bpel.eapi.ExtensionContext;
-import org.apache.ode.bpel.o.OActivity;
-import org.apache.ode.bpel.o.OLink;
-import org.apache.ode.bpel.o.OProcess.OProperty;
-import org.apache.ode.bpel.o.OScope.Variable;
+import org.apache.ode.test.MockExtensionContext;
 import org.apache.ode.utils.DOMUtils;
-import org.apache.ode.utils.SerializableElement;
 import org.junit.Test;
 import org.w3c.dom.Element;
-import org.w3c.dom.Node;
 
 /**
  * @author Tammo van Lessen (University of Stuttgart)
  */
 public class JSOperationTest {
 
-	public static class MockExtensionContext implements ExtensionContext {
-		private Map<String, Node> variables = new HashMap<String, Node>();
-		
-		public Map<String, Node> getVariables() {
-			return variables;
-		}
-		
-		public Long getProcessId() {
-			// TODO Auto-generated method stub
-			return null;
-		}
-
-		public Map<String, Variable> getVisibleVariables()
-				throws FaultException {
-			// TODO Auto-generated method stub
-			return null;
-		}
-
-		public boolean isLinkActive(OLink olink) throws FaultException {
-			// TODO Auto-generated method stub
-			return false;
-		}
-
-		public String readMessageProperty(Variable variable, OProperty property)
-				throws FaultException {
-			// TODO Auto-generated method stub
-			return null;
-		}
-
-		public Node readVariable(Variable variable) throws FaultException {
-			// TODO Auto-generated method stub
-			return null;
-		}
-
-		public Node readVariable(String variableName) throws FaultException {
-			System.out.println("Reading " + variableName);
-			return variables.get(variableName);
-		}
-
-		public void writeVariable(Variable variable, Node value)
-				throws FaultException {
-			// TODO Auto-generated method stub
-			
-		}
-
-		public void writeVariable(String variableName, Node value)
-				throws FaultException {
-			variables.put(variableName, value);
-			System.out.println("Storing in " + variableName + ": " + DOMUtils.domToString(value));
-		}
-
-		public String getActivityName() {
-			return "mockActivity";
-		}
-
-		public OActivity getOActivity() {
-			// TODO Auto-generated method stub
-			return null;
-		}
-	}
-	
 	@Test public void test() throws Exception {
 		StringBuffer s = new StringBuffer();
 		s.append("var request = context.readVariable('request');\n");
-		//s.append("var str = '' + request.TestPart.toString();\n");
-		//s.append("request.TestPart = str + ' World';\n");//
 		s.append("request.TestPart += ' World';\n");
 		s.append("context.writeVariable('request', request);\n");
 
@@ -111,6 +39,6 @@
 		c.getVariables().put("request", DOMUtils.stringToDOM("<message><TestPart>Hello</TestPart></message>"));
 		JSExtensionOperation jso = new JSExtensionOperation();
 		Element e = DOMUtils.stringToDOM("<js:script xmlns:js=\"js\"><![CDATA[" + s + "]]></js:script>");
-		jso.run(c, new SerializableElement(e));
+		jso.run(c, e);
 	}
 }