You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by js...@apache.org on 2008/08/11 17:15:44 UTC

svn commit: r684774 - in /activemq/camel/trunk: camel-core/src/main/java/org/apache/camel/ camel-core/src/main/java/org/apache/camel/component/bean/ camel-core/src/test/java/org/apache/camel/component/bean/ components/camel-spring/src/main/java/org/apa...

Author: jstrachan
Date: Mon Aug 11 08:15:43 2008
New Revision: 684774

URL: http://svn.apache.org/viewvc?rev=684774&view=rev
Log:
added first cut at an implementation of CAMEL-810 to add a @OneWay annotation that can be used to determine if methods should be asynchronous (InOnly)

Added:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/OneWay.java   (contents, props changed)
      - copied, changed from r684733, activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/Header.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/MethodInfoCache.java   (with props)
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanInfoTestTest.java   (with props)
Modified:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanInfo.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/CamelInvocationHandler.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/MethodInfo.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/ProxyHelper.java
    activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/remoting/SendBeforeInterceptor.java

Copied: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/OneWay.java (from r684733, activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/Header.java)
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/OneWay.java?p2=activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/OneWay.java&p1=activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/Header.java&r1=684733&r2=684774&rev=684774&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/Header.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/OneWay.java Mon Aug 11 08:15:43 2008
@@ -22,13 +22,35 @@
 import java.lang.annotation.Target;
 
 /**
- * Marks a parameter as being a header on an inbound {@link Message}
+ * Marks a method as being a one way asynchronous invocation so that if you are using some kind of
+ * <a href="http://activemq.apache.org/camel/spring-remoting.html">Spring Remoting</a> then the method invocation will be asynchronous.
+ *
+ * If you wish to use some other {@link ExchangePattern} than {@link org.apache.camel.ExchangePattern#InOnly} you could use something like
+ *
+ * <code>
+ * @OneWay(ExchangePattern.RobustInOnly)
+ * public void myMethod() {...}
+ * </code>
+ *
+ * otherwise the following code would default to using {@link org.apache.camel.ExchangePattern#InOnly}
+ *
+ * <code>
+ * @OneWay
+ * public void myMethod() {...}
+ * </code>
+ *
+ * @see ExchangePattern
+ * @see Exchange#getPattern()
  *
- * @see Message#getHeader(String) 
  * @version $Revision$
  */
 @Retention(RetentionPolicy.RUNTIME)
-@Target({ElementType.PARAMETER })
-public @interface Header {
-    String name();
-}
+@Target({ElementType.METHOD })
+public @interface OneWay {
+
+    /**
+     * Allows the exact exchange pattern type to be specified though the default value of
+     * {@link org.apache.camel.ExchangePattern#InOnly} should be fine for most uses
+     */
+    public abstract ExchangePattern value() default ExchangePattern.InOnly;
+}
\ No newline at end of file

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/OneWay.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/OneWay.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/OneWay.java
------------------------------------------------------------------------------
    svn:mergeinfo = 

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanInfo.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanInfo.java?rev=684774&r1=684773&r2=684774&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanInfo.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanInfo.java Mon Aug 11 08:15:43 2008
@@ -19,11 +19,7 @@
 import java.lang.annotation.Annotation;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
+import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.camel.Body;
@@ -61,6 +57,7 @@
     private MethodInfo defaultMethod;
     private List<MethodInfo> operationsWithBody = new ArrayList<MethodInfo>();
     private List<MethodInfo> operationsWithCustomAnnotation = new ArrayList<MethodInfo>();
+    private Map<Method, MethodInfo> methodMap = new HashMap<Method, MethodInfo>();
 
     public BeanInfo(CamelContext camelContext, Class type) {
         this(camelContext, type, createParameterMappingStrategy(camelContext));
@@ -131,6 +128,33 @@
     }
 
     protected MethodInfo introspect(Class clazz, Method method) {
+        String opName = method.getName();
+
+        MethodInfo methodInfo = createMethodInfo(clazz, method);
+
+        operations.put(opName, methodInfo);
+        if (methodInfo.hasBodyParameter()) {
+            operationsWithBody.add(methodInfo);
+        }
+        if (methodInfo.isHasCustomAnnotation() && !methodInfo.hasBodyParameter()) {
+            operationsWithCustomAnnotation.add(methodInfo);
+        }
+        methodMap.put(method, methodInfo);
+        return methodInfo;
+    }
+
+    /**
+     * Returns the {@link MethodInfo} for the given method if it exists or null
+     * if there is no metadata available for the given method
+     *
+     * @param method
+     */
+    public MethodInfo getMethodInfo(Method method) {
+        return methodMap.get(method);
+    }
+
+
+    protected MethodInfo createMethodInfo(Class clazz, Method method) {
         Class[] parameterTypes = method.getParameterTypes();
         Annotation[][] parametersAnnotations = method.getParameterAnnotations();
 
@@ -168,7 +192,6 @@
         }
 
         // now lets add the method to the repository
-        String opName = method.getName();
 
         // TODO allow an annotation to expose the operation name to use
         /* if (method.getAnnotation(Operation.class) != null) { String name =
@@ -176,14 +199,6 @@
          * name.length() > 0) { opName = name; } }
          */
         MethodInfo methodInfo = new MethodInfo(clazz, method, parameters, bodyParameters, hasCustomAnnotation);
-
-        operations.put(opName, methodInfo);
-        if (methodInfo.hasBodyParameter()) {
-            operationsWithBody.add(methodInfo);
-        }
-        if (methodInfo.isHasCustomAnnotation() && !methodInfo.hasBodyParameter()) {
-            operationsWithCustomAnnotation.add(methodInfo);
-        }
         return methodInfo;
     }
 

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/CamelInvocationHandler.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/CamelInvocationHandler.java?rev=684774&r1=684773&r2=684774&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/CamelInvocationHandler.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/CamelInvocationHandler.java Mon Aug 11 08:15:43 2008
@@ -16,14 +16,14 @@
  */
 package org.apache.camel.component.bean;
 
-import java.lang.reflect.InvocationHandler;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
-
 import org.apache.camel.Endpoint;
 import org.apache.camel.ExchangePattern;
 import org.apache.camel.Producer;
 
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+
 /**
  * An {@link java.lang.reflect.InvocationHandler} which invokes a
  * message exchange on a camel {@link Endpoint}
@@ -33,15 +33,22 @@
 public class CamelInvocationHandler implements InvocationHandler {
     private final Endpoint endpoint;
     private final Producer producer;
+    private final MethodInfoCache methodInfoCache;
 
-    public CamelInvocationHandler(Endpoint endpoint, Producer producer) {
+    public CamelInvocationHandler(Endpoint endpoint, Producer producer, MethodInfoCache methodInfoCache) {
         this.endpoint = endpoint;
         this.producer = producer;
+        this.methodInfoCache = methodInfoCache;
     }
 
     public Object invoke(Object proxy, Method method, Object[] args) throws Throwable {
         BeanInvocation invocation = new BeanInvocation(method, args);
-        BeanExchange exchange = new BeanExchange(endpoint.getCamelContext(), ExchangePattern.InOut);
+        ExchangePattern pattern = ExchangePattern.InOut;
+        MethodInfo methodInfo = methodInfoCache.getMethodInfo(method);
+        if (methodInfo != null) {
+            pattern = methodInfo.getPattern();
+        }
+        BeanExchange exchange = new BeanExchange(endpoint.getCamelContext(), pattern);
         exchange.setInvocation(invocation);
 
         producer.process(exchange);
@@ -49,7 +56,11 @@
         if (fault != null) {
             throw new InvocationTargetException(fault);
         }
-        return exchange.getOut(true).getBody();
+        if (pattern.isOutCapable()) {
+            return exchange.getOut(true).getBody();
+        } else {
+            return null;
+        }
     }
 }
 

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/MethodInfo.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/MethodInfo.java?rev=684774&r1=684773&r2=684774&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/MethodInfo.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/MethodInfo.java Mon Aug 11 08:15:43 2008
@@ -25,6 +25,8 @@
 
 import org.apache.camel.Exchange;
 import org.apache.camel.Expression;
+import org.apache.camel.ExchangePattern;
+import org.apache.camel.OneWay;
 import org.apache.camel.util.ExchangeHelper;
 import org.apache.camel.util.ObjectHelper;
 
@@ -40,6 +42,7 @@
     private final List<ParameterInfo> bodyParameters;
     private final boolean hasCustomAnnotation;
     private Expression parametersExpression;
+    private ExchangePattern pattern = ExchangePattern.InOut;
 
     public MethodInfo(Class type, Method method, List<ParameterInfo> parameters, List<ParameterInfo> bodyParameters, boolean hasCustomAnnotation) {
         this.type = type;
@@ -48,6 +51,10 @@
         this.bodyParameters = bodyParameters;
         this.hasCustomAnnotation = hasCustomAnnotation;
         this.parametersExpression = createParametersExpression();
+        OneWay oneway = method.getAnnotation(OneWay.class);
+        if (oneway != null) {
+            pattern = oneway.value();
+        }
     }
 
     public String toString() {
@@ -161,4 +168,8 @@
             }
         };
     }
+
+    public ExchangePattern getPattern() {
+        return pattern;
+    }
 }

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/MethodInfoCache.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/MethodInfoCache.java?rev=684774&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/MethodInfoCache.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/MethodInfoCache.java Mon Aug 11 08:15:43 2008
@@ -0,0 +1,82 @@
+/**
+ *
+ * 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.camel.component.bean;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.util.LRUCache;
+
+import java.lang.reflect.Method;
+import java.util.Map;
+
+/**
+ * Represents a cache of MethodInfo objects to avoid the expense of introspection for each invocation of a method
+ * via a proxy
+ *
+ * @version $Revision: 1.1 $
+ */
+public class MethodInfoCache {
+    private final CamelContext camelContext;
+    private Map<Method,MethodInfo> methodCache;
+    private Map<Class,BeanInfo> classCache;
+
+    public MethodInfoCache(CamelContext camelContext) {
+        this(camelContext, 1000, 10000);
+    }
+
+    public MethodInfoCache(CamelContext camelContext, int classCacheSize, int methodCacheSize) {
+        this(camelContext, createLruCache(classCacheSize), createLruCache(methodCacheSize));
+    }
+
+    public MethodInfoCache(CamelContext camelContext, Map<Class, BeanInfo> classCache, Map<Method, MethodInfo> methodCache) {
+        this.camelContext = camelContext;
+        this.classCache = classCache;
+        this.methodCache = methodCache;
+    }
+
+    public synchronized MethodInfo getMethodInfo(Method method) {
+        MethodInfo answer = methodCache.get(method);
+        if (answer == null) {
+            answer = createMethodInfo(method);
+            methodCache.put(method, answer);
+        }
+        return answer;
+    }
+
+    protected  MethodInfo createMethodInfo(Method method) {
+        Class<?> declaringClass = method.getDeclaringClass();
+        BeanInfo info = getBeanInfo(declaringClass);
+        return info.getMethodInfo(method);
+    }
+
+    protected synchronized BeanInfo getBeanInfo(Class<?> declaringClass) {
+        BeanInfo beanInfo = classCache.get(declaringClass);
+        if (beanInfo == null) {
+            beanInfo = createBeanInfo(declaringClass);
+            classCache.put(declaringClass, beanInfo);
+        }
+        return beanInfo;
+    }
+
+    protected BeanInfo createBeanInfo(Class<?> declaringClass) {
+        return new BeanInfo(camelContext, declaringClass);
+    }
+
+    protected static Map createLruCache(int size) {
+        return new LRUCache(size);
+    }
+}

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/MethodInfoCache.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/ProxyHelper.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/ProxyHelper.java?rev=684774&r1=684773&r2=684774&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/ProxyHelper.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/ProxyHelper.java Mon Aug 11 08:15:43 2008
@@ -41,8 +41,18 @@
      */
     public static Object createProxy(final Endpoint endpoint, ClassLoader cl, Class interfaces[])
         throws Exception {
+        MethodInfoCache methodCache = new MethodInfoCache(endpoint.getCamelContext());
+        return createProxy(endpoint, cl, interfaces, methodCache);
+    }
+
+    /**
+     * Creates a Proxy which sends PojoExchange to the endpoint.
+     *
+     * @throws Exception
+     */
+    public static Object createProxy(Endpoint endpoint, ClassLoader cl, Class[] interfaces, MethodInfoCache methodCache) throws Exception {
         final Producer producer = endpoint.createProducer();
-        return Proxy.newProxyInstance(cl, interfaces, new CamelInvocationHandler(endpoint, producer));
+        return Proxy.newProxyInstance(cl, interfaces, new CamelInvocationHandler(endpoint, producer, methodCache));
     }
 
     /**

Added: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanInfoTestTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanInfoTestTest.java?rev=684774&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanInfoTestTest.java (added)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanInfoTestTest.java Mon Aug 11 08:15:43 2008
@@ -0,0 +1,64 @@
+/**
+ *
+ * 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.camel.component.bean;
+
+import junit.framework.TestCase;
+import org.apache.camel.OneWay;
+import org.apache.camel.ExchangePattern;
+import org.apache.camel.CamelContext;
+import org.apache.camel.impl.DefaultCamelContext;
+
+import java.lang.reflect.Method;
+
+/**
+ * @version $Revision: 1.1 $
+ */
+public class BeanInfoTestTest extends TestCase {
+    protected CamelContext camelContext = new DefaultCamelContext();
+    protected BeanInfo info = new BeanInfo(camelContext, Foo.class);
+
+    public void testMethodPattern() throws Exception {
+        assertMethodPattern("inOutMethod", ExchangePattern.InOut);
+        assertMethodPattern("inOnlyMethod", ExchangePattern.InOnly);
+        assertMethodPattern("robustInOnlyMethod", ExchangePattern.RobustInOnly);
+    }
+
+    protected void assertMethodPattern(String methodName, ExchangePattern expectedPattern) throws NoSuchMethodException {
+        Class type = info.getType();
+        Method method = type.getMethod(methodName);
+        assertNotNull("Could not find method: " + methodName, method);
+
+        MethodInfo methodInfo = info.getMethodInfo(method);
+        assertNotNull("Could not find methodInfo for: " + method, methodInfo);
+
+        ExchangePattern actualPattern = methodInfo.getPattern();
+        assertEquals("Pattern for: " + method, expectedPattern, actualPattern);
+
+        //System.out.println("Method: " + method + " has pattern: " + actualPattern);
+    }
+
+    public interface Foo {
+        public void inOutMethod();
+
+        @OneWay
+        public void inOnlyMethod();
+
+        @OneWay(ExchangePattern.RobustInOnly)
+        public void robustInOnlyMethod();
+    }
+}

Propchange: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanInfoTestTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/remoting/SendBeforeInterceptor.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/remoting/SendBeforeInterceptor.java?rev=684774&r1=684773&r2=684774&view=diff
==============================================================================
--- activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/remoting/SendBeforeInterceptor.java (original)
+++ activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/remoting/SendBeforeInterceptor.java Mon Aug 11 08:15:43 2008
@@ -24,6 +24,7 @@
 import org.apache.camel.Endpoint;
 import org.apache.camel.Producer;
 import org.apache.camel.component.bean.CamelInvocationHandler;
+import org.apache.camel.component.bean.MethodInfoCache;
 import org.apache.camel.util.CamelContextHelper;
 
 import org.springframework.beans.factory.DisposableBean;
@@ -54,7 +55,7 @@
         Endpoint endpoint = CamelContextHelper.getMandatoryEndpoint(camelContext, uri);
         producer = endpoint.createProducer();
         producer.start();
-        invocationHandler = new CamelInvocationHandler(endpoint, producer);
+        invocationHandler = new CamelInvocationHandler(endpoint, producer, new MethodInfoCache(endpoint.getCamelContext()));
     }
 
     public void destroy() throws Exception {