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/05/14 22:06:04 UTC

svn commit: r656397 - in /activemq/camel/trunk: camel-core/src/main/java/org/apache/camel/component/bean/ camel-core/src/main/java/org/apache/camel/impl/ components/camel-spring/src/main/java/org/apache/camel/spring/factory/ components/camel-spring/src...

Author: jstrachan
Date: Wed May 14 13:06:04 2008
New Revision: 656397

URL: http://svn.apache.org/viewvc?rev=656397&view=rev
Log:
added improvement for  https://issues.apache.org/activemq/browse/CAMEL-505 to support BeanEndpoint being easily configurable via Spring.xml

Added:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanEndpoint.java   (with props)
    activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/
    activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/BeanEndpointFactory.java   (with props)
    activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest.java   (with props)
    activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest-context.xml   (with props)
Modified:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanComponent.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/RegistryBean.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingEndpoint.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ProcessorEndpoint.java

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanComponent.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanComponent.java?rev=656397&r1=656396&r2=656397&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanComponent.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanComponent.java Wed May 14 13:06:04 2008
@@ -20,6 +20,7 @@
 
 import org.apache.camel.Endpoint;
 import org.apache.camel.ExchangePattern;
+import org.apache.camel.Processor;
 import org.apache.camel.converter.ObjectConverter;
 import org.apache.camel.impl.DefaultComponent;
 import org.apache.camel.impl.ProcessorEndpoint;
@@ -70,21 +71,16 @@
     // Implementation methods
     //-----------------------------------------------------------------------
     protected Endpoint createEndpoint(String uri, String remaining, Map parameters) throws Exception {
-        RegistryBean holder = new RegistryBean(getCamelContext(), remaining);
-        BeanProcessor processor;
-        if (ObjectConverter.toBool(parameters.remove("cache"))) {
-            processor = new BeanProcessor(holder.createCacheHolder());
-        } else {
-            processor = new BeanProcessor(holder);
-        }
+        BeanEndpoint endpoint = new BeanEndpoint(uri, this);
+        endpoint.setBeanName(remaining);
+        endpoint.setCache(ObjectConverter.toBool(parameters.remove("cache")));
+        Processor processor = endpoint.getProcessor();
         setProperties(processor, parameters);
-        return createEndpoint(uri, processor);
+        return endpoint;
     }
 
-    protected ProcessorEndpoint createEndpoint(String uri, BeanProcessor processor) {
-        ProcessorEndpoint answer = new ProcessorEndpoint(uri, this, processor);
-        answer.setExchangePattern(ExchangePattern.InOut);
-        return answer;
+    protected BeanEndpoint createEndpoint(String uri, BeanProcessor processor) {
+        return new BeanEndpoint(uri, this, processor);
     }
 
     protected ParameterMappingStrategy createParameterMappingStrategy() {

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanEndpoint.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanEndpoint.java?rev=656397&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanEndpoint.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanEndpoint.java Wed May 14 13:06:04 2008
@@ -0,0 +1,123 @@
+/**
+ *
+ * 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.Component;
+import org.apache.camel.ExchangePattern;
+import org.apache.camel.Processor;
+import org.apache.camel.impl.ProcessorEndpoint;
+
+/**
+ * @version $Revision: 1.1 $
+ */
+public class BeanEndpoint extends ProcessorEndpoint {
+    private boolean cache;
+    private String beanName;
+    private String method;
+    private BeanHolder beanHolder;
+
+    public BeanEndpoint() {
+        init();
+    }
+
+    public BeanEndpoint(String endpointUri) {
+        super(endpointUri);
+        init();
+    }
+
+    public BeanEndpoint(String endpointUri, BeanProcessor processor) {
+        super(endpointUri, processor);
+        init();
+    }
+
+    public BeanEndpoint(String endpointUri, Component component, BeanProcessor processor) {
+        super(endpointUri, component, processor);
+        init();
+    }
+
+    public BeanEndpoint(String endpointUri, Component component) {
+        super(endpointUri, component);
+        init();
+    }
+
+    // Properties
+    //-------------------------------------------------------------------------
+
+    public String getBeanName() {
+        return beanName;
+    }
+
+    public void setBeanName(String beanName) {
+        this.beanName = beanName;
+    }
+
+    public boolean isCache() {
+        return cache;
+    }
+
+    public void setCache(boolean cache) {
+        this.cache = cache;
+    }
+
+    public String getMethod() {
+        return method;
+    }
+
+    public void setMethod(String method) {
+        this.method = method;
+    }
+
+    public BeanHolder getBeanHolder() {
+        return beanHolder;
+    }
+
+    public void setBeanHolder(BeanHolder beanHolder) {
+        this.beanHolder = beanHolder;
+    }
+
+    // Implementation methods
+    //-------------------------------------------------------------------------
+
+    @Override
+    protected String createEndpointUri() {
+        return "bean:" + getBeanName() + (method != null ? "?method=" + method : "");
+    }
+
+    private void init() {
+        setExchangePattern(ExchangePattern.InOut);
+    }
+
+    @Override
+    protected Processor createProcessor() throws Exception {
+        BeanHolder holder = getBeanHolder();
+        if (holder == null) {
+            RegistryBean registryBean = new RegistryBean(getCamelContext(), beanName);
+            if (cache) {
+                holder = registryBean.createCacheHolder();
+            }
+            else {
+                holder = registryBean;
+            }
+        }
+        BeanProcessor processor = new BeanProcessor(holder);
+        if (method != null) {
+            processor.setMethod(method);
+        }
+        return processor;
+    }
+}

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

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/RegistryBean.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/RegistryBean.java?rev=656397&r1=656396&r2=656397&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/RegistryBean.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/RegistryBean.java Wed May 14 13:06:04 2008
@@ -23,7 +23,7 @@
 import org.apache.camel.util.ObjectHelper;
 
 /**
- * An implementation of a {@link BeanHolder} which will
+ * An implementation of a {@link BeanHolder} which will look up a bean from the registry and act as a cache of its metadata
  *
  * @version $Revision$
  */

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingEndpoint.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingEndpoint.java?rev=656397&r1=656396&r2=656397&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingEndpoint.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingEndpoint.java Wed May 14 13:06:04 2008
@@ -30,6 +30,13 @@
  */
 public abstract class DefaultPollingEndpoint<E extends Exchange> extends ScheduledPollEndpoint<E>  {
 
+    protected DefaultPollingEndpoint() {
+    }
+
+    protected DefaultPollingEndpoint(String endpointUri) {
+        super(endpointUri);
+    }
+
     protected DefaultPollingEndpoint(String endpointUri, Component component) {
         super(endpointUri, component);
     }
@@ -38,10 +45,6 @@
         super(endpointUri, context);
     }
 
-    protected DefaultPollingEndpoint(String endpointUri) {
-        super(endpointUri);
-    }
-
     public Consumer<E> createConsumer(Processor processor) throws Exception {
         DefaultScheduledPollConsumer result = new DefaultScheduledPollConsumer(this, processor);
         configureConsumer(result);

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ProcessorEndpoint.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ProcessorEndpoint.java?rev=656397&r1=656396&r2=656397&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ProcessorEndpoint.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ProcessorEndpoint.java Wed May 14 13:06:04 2008
@@ -31,7 +31,14 @@
  * @version $Revision$
  */
 public class ProcessorEndpoint extends DefaultPollingEndpoint<Exchange> {
-    private final Processor processor;
+    private Processor processor;
+
+    protected ProcessorEndpoint() {
+    }
+
+    protected ProcessorEndpoint(String endpointUri) {
+        super(endpointUri);
+    }
 
     public ProcessorEndpoint(String endpointUri, CamelContext context, Processor processor) {
         super(endpointUri, context);
@@ -48,6 +55,11 @@
         this.processor = processor;
     }
 
+
+    protected ProcessorEndpoint(String endpointUri, Component component) {
+        super(endpointUri, component);
+    }
+
     public Producer<Exchange> createProducer() throws Exception {
         return new DefaultProducer<Exchange>(this) {
             public void process(Exchange exchange) throws Exception {
@@ -61,19 +73,23 @@
         return new ProcessorPollingConsumer(this, getProcessor());
     }
 
-    public Processor getProcessor() {
+    public Processor getProcessor() throws Exception {
         if (processor == null) {
-            return new Processor() {
-                public void process(Exchange exchange) throws Exception {
-                    onExchange(exchange);
-                }
-            };
+            processor = createProcessor();
         }
         return processor;
     }
 
+    protected Processor createProcessor() throws Exception {
+        return new Processor() {
+            public void process(Exchange exchange) throws Exception {
+                onExchange(exchange);
+            }
+        };
+    }
+
     protected void onExchange(Exchange exchange) throws Exception {
-        processor.process(exchange);
+        getProcessor().process(exchange);
     }
 
     public boolean isSingleton() {

Added: activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/BeanEndpointFactory.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/BeanEndpointFactory.java?rev=656397&view=auto
==============================================================================
--- activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/BeanEndpointFactory.java (added)
+++ activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/BeanEndpointFactory.java Wed May 14 13:06:04 2008
@@ -0,0 +1,53 @@
+/**
+ *
+ * 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.spring.factory;
+
+import java.io.File;
+
+import org.springframework.beans.factory.FactoryBean;
+import org.apache.camel.component.file.FileEndpoint;
+import org.apache.camel.component.bean.BeanEndpoint;
+import org.apache.camel.util.ObjectHelper;
+
+/**
+ * @version $Revision: 1.1 $
+ */
+public class BeanEndpointFactory implements FactoryBean {
+    private boolean singleton = true;
+
+    public Object getObject() throws Exception {
+        return new BeanEndpoint();
+    }
+
+    public Class getObjectType() {
+        return FileEndpoint.class;
+    }
+
+    public boolean isSingleton() {
+        return singleton;
+    }
+
+    protected void setSingleton(boolean singleton) {
+        this.singleton = singleton;
+    }
+
+    // Properties
+    //-------------------------------------------------------------------------
+    
+    
+}

Propchange: activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/BeanEndpointFactory.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest.java?rev=656397&view=auto
==============================================================================
--- activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest.java (added)
+++ activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest.java Wed May 14 13:06:04 2008
@@ -0,0 +1,57 @@
+/**
+ *
+ * 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 javax.annotation.Resource;
+
+import org.springframework.test.context.junit38.AbstractJUnit38SpringContextTests;
+import org.springframework.test.context.ContextConfiguration;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.apache.camel.CamelTemplate;
+import org.apache.camel.Endpoint;
+
+/**
+ * @version $Revision: 1.1 $
+ */
+@ContextConfiguration
+public class BeanRouteUsingSpringEndpointTest extends AbstractJUnit38SpringContextTests {
+    @Autowired
+    protected CamelTemplate template;
+    @Resource
+    protected Endpoint helloEndpoint;
+    @Resource
+    protected Endpoint goodbyeEndpoint;
+
+    protected String body = "James";
+
+    public void testSayHello() throws Exception {
+        assertNotNull(helloEndpoint);
+        assertNotNull(goodbyeEndpoint);
+
+        Object value = template.sendBody(helloEndpoint, body);
+
+        assertEquals("Returned value", "Hello James!", value);
+    }
+
+    public void testSayGoodbye() throws Exception {
+        Object value = template.sendBody(goodbyeEndpoint, body);
+
+        assertEquals("Returned value", "Bye James!", value);
+    }
+
+}

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

Added: activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest-context.xml
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest-context.xml?rev=656397&view=auto
==============================================================================
--- activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest-context.xml (added)
+++ activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest-context.xml Wed May 14 13:06:04 2008
@@ -0,0 +1,42 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.0.xsd
+       http://activemq.apache.org/camel/schema/spring http://activemq.apache.org/camel/schema/spring/camel-spring.xsd
+    ">
+
+  <camelContext xmlns="http://activemq.apache.org/camel/schema/spring">
+    <template id="camelTemplate"/>
+  </camelContext>
+
+
+  <bean id="helloEndpoint" class="org.apache.camel.component.bean.BeanEndpoint">
+    <property name="beanName" value="myBean"/>
+    <property name="method" value="sayHello"/>
+  </bean>
+  
+  <bean id="goodbyeEndpoint" class="org.apache.camel.component.bean.BeanEndpoint">
+    <property name="beanName" value="myBean"/>
+    <property name="method" value="sayGoodbye"/>
+  </bean>
+
+  <bean id="myBean" class="org.apache.camel.component.bean.ExampleBean"/>
+
+</beans>

Propchange: activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest-context.xml
------------------------------------------------------------------------------
    svn:eol-style = native



RE: svn commit: r656397 - in /activemq/camel/trunk: camel-core/src/main/java/org/apache/camel/component/bean/ camel-core/src/main/java/org/apache/camel/impl/ components/camel-spring/src/main/java/org/apache/camel/spring/factory/ components/camel-spring/sr

Posted by Claus Ibsen <ci...@silverbullet.dk>.
Hi 

James, isn't there something odd in the BeanEndpointFactory?

getObjectType returns a FileEndpoint and not a BeanEndpoint?


+    public Class getObjectType() {
+        return FileEndpoint.class;
+    }


Med venlig hilsen
 
Claus Ibsen
......................................
Silverbullet
Skovsgårdsvænget 21
8362 Hørning
Tlf. +45 2962 7576
Web: www.silverbullet.dk
-----Original Message-----
From: jstrachan@apache.org [mailto:jstrachan@apache.org] 
Sent: 14. maj 2008 22:06
To: camel-commits@activemq.apache.org
Subject: svn commit: r656397 - in /activemq/camel/trunk: camel-core/src/main/java/org/apache/camel/component/bean/ camel-core/src/main/java/org/apache/camel/impl/ components/camel-spring/src/main/java/org/apache/camel/spring/factory/ components/camel-spring/src...

Author: jstrachan
Date: Wed May 14 13:06:04 2008
New Revision: 656397

URL: http://svn.apache.org/viewvc?rev=656397&view=rev
Log:
added improvement for  https://issues.apache.org/activemq/browse/CAMEL-505 to support BeanEndpoint being easily configurable via Spring.xml

Added:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanEndpoint.java   (with props)
    activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/
    activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/BeanEndpointFactory.java   (with props)
    activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest.java   (with props)
    activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest-context.xml   (with props)
Modified:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanComponent.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/RegistryBean.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingEndpoint.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ProcessorEndpoint.java

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanComponent.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanComponent.java?rev=656397&r1=656396&r2=656397&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanComponent.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanComponent.java Wed May 14 13:06:04 2008
@@ -20,6 +20,7 @@
 
 import org.apache.camel.Endpoint;
 import org.apache.camel.ExchangePattern;
+import org.apache.camel.Processor;
 import org.apache.camel.converter.ObjectConverter;
 import org.apache.camel.impl.DefaultComponent;
 import org.apache.camel.impl.ProcessorEndpoint;
@@ -70,21 +71,16 @@
     // Implementation methods
     //-----------------------------------------------------------------------
     protected Endpoint createEndpoint(String uri, String remaining, Map parameters) throws Exception {
-        RegistryBean holder = new RegistryBean(getCamelContext(), remaining);
-        BeanProcessor processor;
-        if (ObjectConverter.toBool(parameters.remove("cache"))) {
-            processor = new BeanProcessor(holder.createCacheHolder());
-        } else {
-            processor = new BeanProcessor(holder);
-        }
+        BeanEndpoint endpoint = new BeanEndpoint(uri, this);
+        endpoint.setBeanName(remaining);
+        endpoint.setCache(ObjectConverter.toBool(parameters.remove("cache")));
+        Processor processor = endpoint.getProcessor();
         setProperties(processor, parameters);
-        return createEndpoint(uri, processor);
+        return endpoint;
     }
 
-    protected ProcessorEndpoint createEndpoint(String uri, BeanProcessor processor) {
-        ProcessorEndpoint answer = new ProcessorEndpoint(uri, this, processor);
-        answer.setExchangePattern(ExchangePattern.InOut);
-        return answer;
+    protected BeanEndpoint createEndpoint(String uri, BeanProcessor processor) {
+        return new BeanEndpoint(uri, this, processor);
     }
 
     protected ParameterMappingStrategy createParameterMappingStrategy() {

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanEndpoint.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanEndpoint.java?rev=656397&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanEndpoint.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/BeanEndpoint.java Wed May 14 13:06:04 2008
@@ -0,0 +1,123 @@
+/**
+ *
+ * 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.Component;
+import org.apache.camel.ExchangePattern;
+import org.apache.camel.Processor;
+import org.apache.camel.impl.ProcessorEndpoint;
+
+/**
+ * @version $Revision: 1.1 $
+ */
+public class BeanEndpoint extends ProcessorEndpoint {
+    private boolean cache;
+    private String beanName;
+    private String method;
+    private BeanHolder beanHolder;
+
+    public BeanEndpoint() {
+        init();
+    }
+
+    public BeanEndpoint(String endpointUri) {
+        super(endpointUri);
+        init();
+    }
+
+    public BeanEndpoint(String endpointUri, BeanProcessor processor) {
+        super(endpointUri, processor);
+        init();
+    }
+
+    public BeanEndpoint(String endpointUri, Component component, BeanProcessor processor) {
+        super(endpointUri, component, processor);
+        init();
+    }
+
+    public BeanEndpoint(String endpointUri, Component component) {
+        super(endpointUri, component);
+        init();
+    }
+
+    // Properties
+    //-------------------------------------------------------------------------
+
+    public String getBeanName() {
+        return beanName;
+    }
+
+    public void setBeanName(String beanName) {
+        this.beanName = beanName;
+    }
+
+    public boolean isCache() {
+        return cache;
+    }
+
+    public void setCache(boolean cache) {
+        this.cache = cache;
+    }
+
+    public String getMethod() {
+        return method;
+    }
+
+    public void setMethod(String method) {
+        this.method = method;
+    }
+
+    public BeanHolder getBeanHolder() {
+        return beanHolder;
+    }
+
+    public void setBeanHolder(BeanHolder beanHolder) {
+        this.beanHolder = beanHolder;
+    }
+
+    // Implementation methods
+    //-------------------------------------------------------------------------
+
+    @Override
+    protected String createEndpointUri() {
+        return "bean:" + getBeanName() + (method != null ? "?method=" + method : "");
+    }
+
+    private void init() {
+        setExchangePattern(ExchangePattern.InOut);
+    }
+
+    @Override
+    protected Processor createProcessor() throws Exception {
+        BeanHolder holder = getBeanHolder();
+        if (holder == null) {
+            RegistryBean registryBean = new RegistryBean(getCamelContext(), beanName);
+            if (cache) {
+                holder = registryBean.createCacheHolder();
+            }
+            else {
+                holder = registryBean;
+            }
+        }
+        BeanProcessor processor = new BeanProcessor(holder);
+        if (method != null) {
+            processor.setMethod(method);
+        }
+        return processor;
+    }
+}

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

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/RegistryBean.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/RegistryBean.java?rev=656397&r1=656396&r2=656397&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/RegistryBean.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/bean/RegistryBean.java Wed May 14 13:06:04 2008
@@ -23,7 +23,7 @@
 import org.apache.camel.util.ObjectHelper;
 
 /**
- * An implementation of a {@link BeanHolder} which will
+ * An implementation of a {@link BeanHolder} which will look up a bean from the registry and act as a cache of its metadata
  *
  * @version $Revision$
  */

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingEndpoint.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingEndpoint.java?rev=656397&r1=656396&r2=656397&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingEndpoint.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingEndpoint.java Wed May 14 13:06:04 2008
@@ -30,6 +30,13 @@
  */
 public abstract class DefaultPollingEndpoint<E extends Exchange> extends ScheduledPollEndpoint<E>  {
 
+    protected DefaultPollingEndpoint() {
+    }
+
+    protected DefaultPollingEndpoint(String endpointUri) {
+        super(endpointUri);
+    }
+
     protected DefaultPollingEndpoint(String endpointUri, Component component) {
         super(endpointUri, component);
     }
@@ -38,10 +45,6 @@
         super(endpointUri, context);
     }
 
-    protected DefaultPollingEndpoint(String endpointUri) {
-        super(endpointUri);
-    }
-
     public Consumer<E> createConsumer(Processor processor) throws Exception {
         DefaultScheduledPollConsumer result = new DefaultScheduledPollConsumer(this, processor);
         configureConsumer(result);

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ProcessorEndpoint.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ProcessorEndpoint.java?rev=656397&r1=656396&r2=656397&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ProcessorEndpoint.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/ProcessorEndpoint.java Wed May 14 13:06:04 2008
@@ -31,7 +31,14 @@
  * @version $Revision$
  */
 public class ProcessorEndpoint extends DefaultPollingEndpoint<Exchange> {
-    private final Processor processor;
+    private Processor processor;
+
+    protected ProcessorEndpoint() {
+    }
+
+    protected ProcessorEndpoint(String endpointUri) {
+        super(endpointUri);
+    }
 
     public ProcessorEndpoint(String endpointUri, CamelContext context, Processor processor) {
         super(endpointUri, context);
@@ -48,6 +55,11 @@
         this.processor = processor;
     }
 
+
+    protected ProcessorEndpoint(String endpointUri, Component component) {
+        super(endpointUri, component);
+    }
+
     public Producer<Exchange> createProducer() throws Exception {
         return new DefaultProducer<Exchange>(this) {
             public void process(Exchange exchange) throws Exception {
@@ -61,19 +73,23 @@
         return new ProcessorPollingConsumer(this, getProcessor());
     }
 
-    public Processor getProcessor() {
+    public Processor getProcessor() throws Exception {
         if (processor == null) {
-            return new Processor() {
-                public void process(Exchange exchange) throws Exception {
-                    onExchange(exchange);
-                }
-            };
+            processor = createProcessor();
         }
         return processor;
     }
 
+    protected Processor createProcessor() throws Exception {
+        return new Processor() {
+            public void process(Exchange exchange) throws Exception {
+                onExchange(exchange);
+            }
+        };
+    }
+
     protected void onExchange(Exchange exchange) throws Exception {
-        processor.process(exchange);
+        getProcessor().process(exchange);
     }
 
     public boolean isSingleton() {

Added: activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/BeanEndpointFactory.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/BeanEndpointFactory.java?rev=656397&view=auto
==============================================================================
--- activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/BeanEndpointFactory.java (added)
+++ activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/BeanEndpointFactory.java Wed May 14 13:06:04 2008
@@ -0,0 +1,53 @@
+/**
+ *
+ * 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.spring.factory;
+
+import java.io.File;
+
+import org.springframework.beans.factory.FactoryBean;
+import org.apache.camel.component.file.FileEndpoint;
+import org.apache.camel.component.bean.BeanEndpoint;
+import org.apache.camel.util.ObjectHelper;
+
+/**
+ * @version $Revision: 1.1 $
+ */
+public class BeanEndpointFactory implements FactoryBean {
+    private boolean singleton = true;
+
+    public Object getObject() throws Exception {
+        return new BeanEndpoint();
+    }
+
+    public Class getObjectType() {
+        return FileEndpoint.class;
+    }
+
+    public boolean isSingleton() {
+        return singleton;
+    }
+
+    protected void setSingleton(boolean singleton) {
+        this.singleton = singleton;
+    }
+
+    // Properties
+    //-------------------------------------------------------------------------
+    
+    
+}

Propchange: activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/factory/BeanEndpointFactory.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest.java?rev=656397&view=auto
==============================================================================
--- activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest.java (added)
+++ activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest.java Wed May 14 13:06:04 2008
@@ -0,0 +1,57 @@
+/**
+ *
+ * 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 javax.annotation.Resource;
+
+import org.springframework.test.context.junit38.AbstractJUnit38SpringContextTests;
+import org.springframework.test.context.ContextConfiguration;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.apache.camel.CamelTemplate;
+import org.apache.camel.Endpoint;
+
+/**
+ * @version $Revision: 1.1 $
+ */
+@ContextConfiguration
+public class BeanRouteUsingSpringEndpointTest extends AbstractJUnit38SpringContextTests {
+    @Autowired
+    protected CamelTemplate template;
+    @Resource
+    protected Endpoint helloEndpoint;
+    @Resource
+    protected Endpoint goodbyeEndpoint;
+
+    protected String body = "James";
+
+    public void testSayHello() throws Exception {
+        assertNotNull(helloEndpoint);
+        assertNotNull(goodbyeEndpoint);
+
+        Object value = template.sendBody(helloEndpoint, body);
+
+        assertEquals("Returned value", "Hello James!", value);
+    }
+
+    public void testSayGoodbye() throws Exception {
+        Object value = template.sendBody(goodbyeEndpoint, body);
+
+        assertEquals("Returned value", "Bye James!", value);
+    }
+
+}

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

Added: activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest-context.xml
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest-context.xml?rev=656397&view=auto
==============================================================================
--- activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest-context.xml (added)
+++ activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest-context.xml Wed May 14 13:06:04 2008
@@ -0,0 +1,42 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.0.xsd
+       http://activemq.apache.org/camel/schema/spring http://activemq.apache.org/camel/schema/spring/camel-spring.xsd
+    ">
+
+  <camelContext xmlns="http://activemq.apache.org/camel/schema/spring">
+    <template id="camelTemplate"/>
+  </camelContext>
+
+
+  <bean id="helloEndpoint" class="org.apache.camel.component.bean.BeanEndpoint">
+    <property name="beanName" value="myBean"/>
+    <property name="method" value="sayHello"/>
+  </bean>
+  
+  <bean id="goodbyeEndpoint" class="org.apache.camel.component.bean.BeanEndpoint">
+    <property name="beanName" value="myBean"/>
+    <property name="method" value="sayGoodbye"/>
+  </bean>
+
+  <bean id="myBean" class="org.apache.camel.component.bean.ExampleBean"/>
+
+</beans>

Propchange: activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/component/bean/BeanRouteUsingSpringEndpointTest-context.xml
------------------------------------------------------------------------------
    svn:eol-style = native