You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by da...@apache.org on 2012/04/05 13:35:26 UTC

svn commit: r1309779 - in /camel/trunk/camel-core/src: main/java/org/apache/camel/component/bean/ test/java/org/apache/camel/component/bean/

Author: davsclaus
Date: Thu Apr  5 11:35:25 2012
New Revision: 1309779

URL: http://svn.apache.org/viewvc?rev=1309779&view=rev
Log:
CAMEL-5140: bean component - @Handler should take precedence in a bean that implements Predicate

Added:
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanHandlerMethodPredicateTest.java
      - copied, changed from r1309668, camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanHandlerMethodTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/MyHandlerPredicateBean.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanProcessor.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanProcessor.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanProcessor.java?rev=1309779&r1=1309778&r2=1309779&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanProcessor.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanProcessor.java Thu Apr  5 11:35:25 2012
@@ -89,17 +89,19 @@ public class BeanProcessor extends Servi
         }
 
         // do we have a custom adapter for this POJO to a Processor
-        // should not be invoked if an explicit method has been set
-        Processor processor = getProcessor();
-        if (explicitMethodName == null && processor != null) {
-            LOG.trace("Using a custom adapter as bean invocation: {}", processor);
-            try {
-                processor.process(exchange);
-            } catch (Throwable e) {
-                exchange.setException(e);
+        // but only do this if allowed
+        if (allowProcessor(explicitMethodName, beanInfo)) {
+            Processor processor = getProcessor();
+            if (processor != null) {
+                LOG.trace("Using a custom adapter as bean invocation: {}", processor);
+                try {
+                    processor.process(exchange);
+                } catch (Throwable e) {
+                    exchange.setException(e);
+                }
+                callback.done(true);
+                return true;
             }
-            callback.done(true);
-            return true;
         }
 
         Message in = exchange.getIn();
@@ -253,4 +255,23 @@ public class BeanProcessor extends Servi
     protected void doStop() throws Exception {
         ServiceHelper.stopService(getProcessor());
     }
+
+    private boolean allowProcessor(String explicitMethodName, BeanInfo info) {
+        if (explicitMethodName != null) {
+            // don't allow if explicit method name is given, as we then must invoke this method
+            return false;
+        }
+
+        // don't allow if any of the methods has a @Handler annotation
+        // as the @Handler annotation takes precedence and is supposed to trigger invocation
+        // of the given method
+        for (MethodInfo method : info.getMethods()) {
+            if (method.hasHandlerAnnotation()) {
+                return false;
+            }
+        }
+
+        // fallback and allow using the processor
+        return true;
+    }
 }

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanHandlerMethodPredicateTest.java (from r1309668, camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanHandlerMethodTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanHandlerMethodPredicateTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanHandlerMethodPredicateTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanHandlerMethodTest.java&r1=1309668&r2=1309779&rev=1309779&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanHandlerMethodTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/BeanHandlerMethodPredicateTest.java Thu Apr  5 11:35:25 2012
@@ -16,141 +16,31 @@
  */
 package org.apache.camel.component.bean;
 
-import org.apache.camel.Body;
 import org.apache.camel.ContextTestSupport;
-import org.apache.camel.Exchange;
-import org.apache.camel.Handler;
-import org.apache.camel.impl.DefaultExchange;
+import org.apache.camel.builder.RouteBuilder;
 
 /**
  * @version 
  */
-public class BeanHandlerMethodTest extends ContextTestSupport {
-
-    public void testNoHandleMethod() throws Exception {
-        BeanInfo info = new BeanInfo(context, MyNoDummyBean.class);
-
-        Exchange exchange = new DefaultExchange(context);
-        MyNoDummyBean pojo = new MyNoDummyBean();
-        MethodInvocation mi = info.createInvocation(pojo, exchange);
-        assertNotNull(mi);
-        assertEquals("hello", mi.getMethod().getName());
-    }
-
-    public void testAmbigiousMethod() throws Exception {
-        BeanInfo info = new BeanInfo(context, MyAmbigiousBean.class);
-
-        Exchange exchange = new DefaultExchange(context);
-        MyAmbigiousBean pojo = new MyAmbigiousBean();
-        try {
-            info.createInvocation(pojo, exchange);
-            fail("Should have thrown an exception");
-        } catch (AmbiguousMethodCallException e) {
-            assertEquals(2, e.getMethods().size());
-        }
-    }
+public class BeanHandlerMethodPredicateTest extends ContextTestSupport {
 
     public void testHandleMethod() throws Exception {
-        BeanInfo info = new BeanInfo(context, MyDummyBean.class);
-
-        Exchange exchange = new DefaultExchange(context);
-        MyDummyBean pojo = new MyDummyBean();
-        MethodInvocation mi = info.createInvocation(pojo, exchange);
-        assertNotNull(mi);
-        assertEquals("hello", mi.getMethod().getName());
-    }
-
-    public void testHandleAndBodyMethod() throws Exception {
-        BeanInfo info = new BeanInfo(context, MyOtherDummyBean.class);
-
-        Exchange exchange = new DefaultExchange(context);
-        MyOtherDummyBean pojo = new MyOtherDummyBean();
-        MethodInvocation mi = info.createInvocation(pojo, exchange);
-        assertNotNull(mi);
-        assertEquals("hello", mi.getMethod().getName());
-    }
-
-    public void testHandleAmbigious() throws Exception {
-        BeanInfo info = new BeanInfo(context, MyReallyDummyBean.class);
-
-        Exchange exchange = new DefaultExchange(context);
-        MyReallyDummyBean pojo = new MyReallyDummyBean();
-        try {
-            info.createInvocation(pojo, exchange);
-            fail("Should throw exception");
-        } catch (AmbiguousMethodCallException e) {
-            assertEquals(2, e.getMethods().size());
-        }
-    }
-
-    public static class MyNoDummyBean {
+        getMockEndpoint("mock:result").expectedBodiesReceived("Hi Camel");
 
-        public String hello(@Body String hi) {
-            return "Hello " + hi;
-        }
-
-        public String doCompute(String input) {
-            fail("Should not invoke me");
-            return null;
-        }
+        template.sendBody("direct:start", "Camel");
 
+        assertMockEndpointsSatisfied();
     }
 
-    public static class MyAmbigiousBean {
-
-        public String hello(String hi) {
-            fail("Should not invoke me");
-            return "Hello " + hi;
-        }
-
-        public String doCompute(String input) {
-            fail("Should not invoke me");
-            return null;
-        }
-
-    }
-
-    public static class MyDummyBean {
-
-        @Handler
-        public String hello(String hi) {
-            return "Hello " + hi;
-        }
-
-        public String doCompute(String input) {
-            fail("Should not invoke me");
-            return null;
-        }
-
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .bean(MyHandlerPredicateBean.class)
+                    .to("mock:result");
+            }
+        };
     }
-
-    public static class MyOtherDummyBean {
-
-        @Handler
-        public String hello(String hi) {
-            return "Hello " + hi;
-        }
-
-        public String bye(@Body String input) {
-            fail("Should not invoke me");
-            return null;
-        }
-
-    }
-
-    public static class MyReallyDummyBean {
-
-        @Handler
-        public String hello(String hi) {
-            return "Hello " + hi;
-        }
-
-        @Handler
-        public String bye(@Body String input) {
-            fail("Should not invoke me");
-            return null;
-        }
-
-    }
-
 }
\ No newline at end of file

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/MyHandlerPredicateBean.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/MyHandlerPredicateBean.java?rev=1309779&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/MyHandlerPredicateBean.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/component/bean/MyHandlerPredicateBean.java Thu Apr  5 11:35:25 2012
@@ -0,0 +1,41 @@
+/**
+ * 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.Exchange;
+import org.apache.camel.Handler;
+import org.apache.camel.Predicate;
+
+/**
+ *
+ */
+public class MyHandlerPredicateBean implements Predicate {
+
+    @Override
+    public boolean matches(Exchange exchange) {
+        return true;
+    }
+
+    @Handler
+    public String doSomething(String body) {
+        return "Hi " + body;
+    }
+
+    public String doSomethingElse(String body) {
+        return "Bye " + body;
+    }
+}