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 2008/12/08 12:55:05 UTC

svn commit: r724319 - in /activemq/camel/trunk/camel-core/src: main/java/org/apache/camel/component/file/ main/java/org/apache/camel/impl/ test/java/org/apache/camel/component/file/ test/java/org/apache/camel/impl/

Author: davsclaus
Date: Mon Dec  8 03:55:04 2008
New Revision: 724319

URL: http://svn.apache.org/viewvc?rev=724319&view=rev
Log:
CAMEL-895: Added support for # notation in endpoint URIs to lookup a bean in the registry

Added:
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/DefaultComponentReferencePropertiesTest.java   (contents, props changed)
      - copied, changed from r723863, activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/DefaultComponentTest.java
Modified:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileComponent.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileConsumer.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileEndpoint.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFileFilterTest.java
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentRefTest.java
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileSorterRefTest.java

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileComponent.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileComponent.java?rev=724319&r1=724318&r2=724319&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileComponent.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileComponent.java Mon Dec  8 03:55:04 2008
@@ -17,7 +17,6 @@
 package org.apache.camel.component.file;
 
 import java.io.File;
-import java.io.FileFilter;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.Map;
@@ -25,8 +24,8 @@
 import org.apache.camel.CamelContext;
 import org.apache.camel.Endpoint;
 import org.apache.camel.impl.DefaultComponent;
-import org.apache.camel.spi.IdempotentRepository;
 import org.apache.camel.util.ObjectHelper;
+import static org.apache.camel.util.ObjectHelper.isNotNullAndNonEmpty;
 
 /**
  * The <a href="http://activemq.apache.org/camel/file.html">File Component</a>
@@ -68,35 +67,14 @@
         File file = new File(remaining);
         FileEndpoint result = new FileEndpoint(file, uri, this);
 
-        // lookup idempotent repository in registry if provided
-        String ref = getAndRemoveParameter(parameters, "idempotentRepositoryRef", String.class);
-        if (ref != null) {
-            IdempotentRepository repository = mandatoryLookup(ref, IdempotentRepository.class);
-            result.setIdempotentRepository(repository);
-        }
-
-        // lookup file filter in registry if provided
-        ref = getAndRemoveParameter(parameters, "fileFilterRef", String.class);
-        if (ref != null) {
-            FileFilter filter = mandatoryLookup(ref, FileFilter.class);
-            result.setFilter(filter);
-        }
-
-        // lookup sorter in registry if provided
-        ref = getAndRemoveParameter(parameters, "sorterRef", String.class);
-        if (ref != null) {
-            Comparator<File> sorter = mandatoryLookup(ref, Comparator.class);
-            result.setFileSorter(sorter);
-        }
-
         // sort by using file language 
         String sortBy = getAndRemoveParameter(parameters, "sortBy", String.class);
-        if (sortBy != null) {
+        if (isNotNullAndNonEmpty(sortBy) && !isReferenceParameter(sortBy)) {
             // we support nested sort groups so they should be chained
             String[] groups = sortBy.split(";");
             Iterator<String> it = ObjectHelper.createIterator(groups);
             Comparator<FileExchange> comparator = createSortByComparator(it);
-            result.setExchangeSorter(comparator);
+            result.setSortBy(comparator);
         }
 
         setProperties(result, parameters);

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileConsumer.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileConsumer.java?rev=724319&r1=724318&r2=724319&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileConsumer.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileConsumer.java Mon Dec  8 03:55:04 2008
@@ -51,14 +51,14 @@
         this.endpoint = endpoint;
     }
 
-    protected synchronized void poll() throws Exception {
+    protected void poll() throws Exception {
         // gather list of files to process
         List<File> files = new ArrayList<File>();
         scanFilesToPoll(endpoint.getFile(), true, files);
 
         // sort files using file comparator if provided
-        if (endpoint.getFileSorter() != null) {
-            Collections.sort(files, endpoint.getFileSorter());
+        if (endpoint.getSorter() != null) {
+            Collections.sort(files, endpoint.getSorter());
         }
 
         // sort using build in sorters that is expression based
@@ -70,8 +70,8 @@
             exchanges.add(exchange);
         }
         // sort files using exchange comparator if provided
-        if (endpoint.getExchangeSorter() != null) {
-            Collections.sort(exchanges, endpoint.getExchangeSorter());
+        if (endpoint.getSortBy() != null) {
+            Collections.sort(exchanges, endpoint.getSortBy());
         }
 
         // consume files one by one

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileEndpoint.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileEndpoint.java?rev=724319&r1=724318&r2=724319&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileEndpoint.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/component/file/FileEndpoint.java Mon Dec  8 03:55:04 2008
@@ -76,8 +76,8 @@
     private boolean idempotent;
     private IdempotentRepository idempotentRepository;
     private FileFilter filter;
-    private Comparator<File> fileSorter;
-    private Comparator<FileExchange> exchangeSorter;
+    private Comparator<File> sorter;
+    private Comparator<FileExchange> sortBy;
 
     protected FileEndpoint(File file, String endpointUri, FileComponent component) {
         super(endpointUri, component);
@@ -400,28 +400,28 @@
         this.filter = filter;
     }
 
-    public Comparator<File> getFileSorter() {
-        return fileSorter;
+    public Comparator<File> getSorter() {
+        return sorter;
     }
 
-    public void setFileSorter(Comparator<File> fileSorter) {
-        this.fileSorter = fileSorter;
+    public void setSorter(Comparator<File> sorter) {
+        this.sorter = sorter;
     }
 
-    public Comparator<FileExchange> getExchangeSorter() {
-        return exchangeSorter;
+    public Comparator<FileExchange> getSortBy() {
+        return sortBy;
     }
 
-    public void setExchangeSorter(Comparator<FileExchange> exchangeSorter) {
-        this.exchangeSorter = exchangeSorter;
+    public void setSortBy(Comparator<FileExchange> sortBy) {
+        this.sortBy = sortBy;
     }
 
-    public void setExchangeSorter(String expression) {
-        setExchangeSorter(expression, false);
+    public void setSortBy(String expression) {
+        setSortBy(expression, false);
     }
 
-    public void setExchangeSorter(String expression, boolean reverse) {
-        setExchangeSorter(DefaultFileSorter.sortByFileLanguage(expression, reverse));
+    public void setSortBy(String expression, boolean reverse) {
+        setSortBy(DefaultFileSorter.sortByFileLanguage(expression, reverse));
     }
 
     /**

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java?rev=724319&r1=724318&r2=724319&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java Mon Dec  8 03:55:04 2008
@@ -17,6 +17,7 @@
 package org.apache.camel.impl;
 
 import java.net.URI;
+import java.util.Iterator;
 import java.util.Map;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
@@ -185,7 +186,48 @@
      * Sets the bean properties on the given bean
      */
     protected void setProperties(Object bean, Map parameters) throws Exception {
-        IntrospectionSupport.setProperties(getCamelContext().getTypeConverter(), bean, parameters);
+        if (useIntrospectionOnEndpoint()) {
+            // set reference properties first as they use # syntax that fools the regular properties setter
+            setReferenceProperties(bean, parameters);
+            IntrospectionSupport.setProperties(getCamelContext().getTypeConverter(), bean, parameters);
+        }
+    }
+
+    /**
+     * Sets the reference properties on the given bean
+     * <p/>
+     * This is convention over configuration, setting all reference parameters (identifier with a value starting with #)
+     * by looking it up in registry and setting it on the bean if possible.
+     */
+    protected void setReferenceProperties(Object bean, Map parameters) throws Exception {
+        Iterator it = parameters.keySet().iterator();
+        while (it.hasNext()) {
+            Object key = it.next();
+            String value = (String) parameters.get(key);
+            if (isReferenceParameter(value)) {
+                Object ref = lookup(value.substring(1));
+                String name = key.toString();
+                if (ref != null) {
+                    boolean hit = IntrospectionSupport.setProperty(getCamelContext().getTypeConverter(), bean, name, ref);
+                    if (hit) {
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("Configued property: " + name + " on bean: " + bean + " with value: " + ref);
+                        }
+                        // must remove as its a valid option and we could configure it
+                        it.remove();
+                    } else {
+                        throw new IllegalArgumentException("Property: " + name + " not found on bean: " + bean + " of type: " + bean.getClass().getName());
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * Is the given parameter a reference parameter
+     */
+    protected boolean isReferenceParameter(String parameter) {
+        return parameter != null && parameter.startsWith("#");
     }
 
     /**
@@ -258,7 +300,7 @@
     /**
      * Gets the parameter and remove it from the parameter map.
      * 
-     * @param parameters  the parameters
+     * @param parameters the parameters
      * @param key        the key
      * @param type       the requested type to convert the value from the parameter
      * @return  the converted value parameter, <tt>null</tt> if parameter does not exists.
@@ -270,7 +312,7 @@
     /**
      * Gets the parameter and remove it from the parameter map.
      *
-     * @param parameters     the parameters
+     * @param parameters    the parameters
      * @param key           the key
      * @param type          the requested type to convert the value from the parameter
      * @param defaultValue  use this default value if the parameter does not contain the key
@@ -305,5 +347,5 @@
         }
         return null;
     }
-    
+
 }

Modified: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFileFilterTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFileFilterTest.java?rev=724319&r1=724318&r2=724319&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFileFilterTest.java (original)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerFileFilterTest.java Mon Dec  8 03:55:04 2008
@@ -29,7 +29,7 @@
  */
 public class FileConsumerFileFilterTest extends ContextTestSupport {
 
-    private String fileUrl = "file://target/filefilter/?fileFilterRef=myFilter";
+    private String fileUrl = "file://target/filefilter/?filter=#myFilter";
 
     @Override
     protected JndiRegistry createRegistry() throws Exception {

Modified: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentRefTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentRefTest.java?rev=724319&r1=724318&r2=724319&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentRefTest.java (original)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileConsumerIdempotentRefTest.java Mon Dec  8 03:55:04 2008
@@ -49,7 +49,7 @@
     protected RouteBuilder createRouteBuilder() throws Exception {
         return new RouteBuilder() {
             public void configure() throws Exception {
-                from("file://target/idempotent/?idempotent=true&idempotentRepositoryRef=myRepo&moveNamePrefix=done/").to("mock:result");
+                from("file://target/idempotent/?idempotent=true&idempotentRepository=#myRepo&moveNamePrefix=done/").to("mock:result");
             }
         };
     }

Modified: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileSorterRefTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileSorterRefTest.java?rev=724319&r1=724318&r2=724319&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileSorterRefTest.java (original)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/component/file/FileSorterRefTest.java Mon Dec  8 03:55:04 2008
@@ -29,7 +29,7 @@
  */
 public class FileSorterRefTest extends ContextTestSupport {
 
-    private String fileUrl = "file://target/filesorter/?sorterRef=mySorter";
+    private String fileUrl = "file://target/filesorter/?sorter=#mySorter";
 
     @Override
     protected JndiRegistry createRegistry() throws Exception {

Copied: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/DefaultComponentReferencePropertiesTest.java (from r723863, activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/DefaultComponentTest.java)
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/DefaultComponentReferencePropertiesTest.java?p2=activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/DefaultComponentReferencePropertiesTest.java&p1=activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/DefaultComponentTest.java&r1=723863&r2=724319&rev=724319&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/DefaultComponentTest.java (original)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/DefaultComponentReferencePropertiesTest.java Mon Dec  8 03:55:04 2008
@@ -16,75 +16,160 @@
  */
 package org.apache.camel.impl;
 
-import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.camel.CamelContext;
+import org.apache.camel.Component;
+import org.apache.camel.Consumer;
 import org.apache.camel.ContextTestSupport;
 import org.apache.camel.Endpoint;
+import org.apache.camel.Exchange;
+import org.apache.camel.Expression;
+import org.apache.camel.Processor;
+import org.apache.camel.Producer;
+import org.apache.camel.ResolveEndpointFailedException;
+import org.apache.camel.builder.ExpressionBuilder;
 
 /**
- * Unit test for helper methods on the DefaultComponent.
+ * Unit test for reference properties
  */
-public class DefaultComponentTest extends ContextTestSupport {
+public class DefaultComponentReferencePropertiesTest extends ContextTestSupport {
 
-    private final class MyComponent extends DefaultComponent {
+    public final class MyEndpoint extends DefaultEndpoint {
 
-        private MyComponent(CamelContext context) {
-            super(context);
+        private Expression expression;
+        private String stringExpression;
+        private String name;
+        private Expression special;
+
+        private MyEndpoint(String endpointUri, Component component) {
+            super(endpointUri, component);
         }
 
-        protected Endpoint createEndpoint(String uri, String remaining, Map parameters)
-            throws Exception {
+        public boolean isSingleton() {
+            return true;
+        }
+
+        public Producer createProducer() throws Exception {
             return null;
         }
-    }
 
-    public void testGetAndRemoveParameterEmptyMap() {
-        Map parameters = new HashMap();
-        MyComponent my = new MyComponent(this.context);
-        Integer value = my.getAndRemoveParameter(parameters, "size", Integer.class);
-        assertNull(value);
-    }
+        public Consumer createConsumer(Processor processor) throws Exception {
+            return null;
+        }
+
+        public void setExpression(Expression expression) {
+            this.expression = expression;
+        }
+
+        public void setExpression(String expression) {
+            stringExpression = expression;
+        }
 
-    public void testGetAndRemoveParameterEmptyMapDefault() {
-        Map parameters = new HashMap();
-        MyComponent my = new MyComponent(this.context);
-        Integer value = my.getAndRemoveParameter(parameters, "size", Integer.class, 5);
-        assertEquals(value.intValue(), 5);
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        public void setSpecial(Expression special) {
+            this.special = special;
+        }
     }
 
-    public void testGetAndRemoveParameterEmptyMapDefaultIsNull() {
-        Map parameters = new HashMap();
-        MyComponent my = new MyComponent(this.context);
-        Integer value = my.getAndRemoveParameter(parameters, "size", Integer.class, null);
-        assertNull(value);
+    public final class MyComponent extends DefaultComponent {
+
+        private MyComponent(CamelContext context) {
+            super(context);
+        }
+
+        protected Endpoint createEndpoint(String uri, String remaining, Map parameters) throws Exception {
+            MyEndpoint result = new MyEndpoint(uri, this);
+            setProperties(result, parameters);
+            return result;
+        }
+
     }
 
-    public void testGetAndRemoveParameterToInteger() {
-        Map parameters = new HashMap();
-        parameters.put("size", 200);
-        MyComponent my = new MyComponent(this.context);
-        Integer value = my.getAndRemoveParameter(parameters, "size", Integer.class);
-        assertEquals(value.intValue(), 200);
+    @Override
+    protected JndiRegistry createRegistry() throws Exception {
+        JndiRegistry jndi = super.createRegistry();
+        jndi.bind("myExpression", ExpressionBuilder.bodyExpression());
+        return jndi;
+    }
+
+    public void testOnlyStringSetter() throws Exception {
+        MyComponent component = new MyComponent(context);
+        MyEndpoint endpoint = (MyEndpoint) component.createEndpoint("foo://?name=Claus");
+        assertNotNull(endpoint);
+        assertEquals("Claus", endpoint.name);
+        assertNull(endpoint.expression);
+        assertNull(endpoint.stringExpression);
+    }
+
+    public void testCallStringSetter() throws Exception {
+        MyComponent component = new MyComponent(context);
+        MyEndpoint endpoint = (MyEndpoint) component.createEndpoint("foo://?expression=hello");
+        assertNotNull(endpoint);
+        assertEquals("hello", endpoint.stringExpression);
+        assertNull(endpoint.expression);
+        assertNull(endpoint.name);
+    }
+
+    public void testNoBeanInRegistryThenCallStringSetter() throws Exception {
+        MyComponent component = new MyComponent(context);
+        MyEndpoint endpoint = (MyEndpoint) component.createEndpoint("foo://?expression=#hello");
+        assertNotNull(endpoint);
+        assertEquals("#hello", endpoint.stringExpression);
+        assertNull(endpoint.expression);
+        assertNull(endpoint.name);
+    }
+
+    public void testCallExpressionSetter() throws Exception {
+        MyComponent component = new MyComponent(context);
+        MyEndpoint endpoint = (MyEndpoint) component.createEndpoint("foo://?expression=#myExpression");
+        assertNotNull(endpoint);
+
+        assertNull(endpoint.stringExpression);
+        assertNotNull(endpoint.expression);
+        assertNull(endpoint.name);
+
+        Exchange exchange = new DefaultExchange(context);
+        exchange.getIn().setBody("Hello World");
+        assertEquals("Hello World", endpoint.expression.evaluate(exchange));
+    }
+
+    public void testCallSingleExpressionSetter() throws Exception {
+        MyComponent component = new MyComponent(context);
+        MyEndpoint endpoint = (MyEndpoint) component.createEndpoint("foo://?special=#myExpression");
+        assertNotNull(endpoint);
+
+        assertNull(endpoint.stringExpression);
+        assertNull(endpoint.expression);
+        assertNull(endpoint.name);
+        assertNotNull(endpoint.special);
+
+        Exchange exchange = new DefaultExchange(context);
+        exchange.getIn().setBody("Hello World");
+        assertEquals("Hello World", endpoint.special.evaluate(exchange));
     }
 
-    public void testGetAndRemoveParameterToIntegerDefault() {
-        Map parameters = new HashMap();
-        parameters.put("size", 200);
-        MyComponent my = new MyComponent(this.context);
-        Integer value = my.getAndRemoveParameter(parameters, "level", Integer.class, 4);
-        assertEquals(value.intValue(), 4);
+    public void testTypoInParameter() throws Exception {
+        MyComponent component = new MyComponent(context);
+        try {
+            component.createEndpoint("foo://?xxxexpression=#hello");
+            fail("Should have throw a ResolveEndpointFailedException");
+        } catch (ResolveEndpointFailedException e) {
+            // ok
+        }
     }
 
-    public void testContextShouldBeSet() throws Exception {
-        MyComponent my = new MyComponent(null);
+    public void testTypoInParameterValue() throws Exception {
+        MyComponent component = new MyComponent(context);
         try {
-            my.start();
-            fail("Should have thrown a IllegalArgumentException");
+            component.createEndpoint("foo://?special=#dummy");
+            fail("Should have throw a IllegalArgumentException");
         } catch (IllegalArgumentException e) {
-            assertEquals("camelContext must be specified", e.getMessage());
+            // ok
         }
     }
 
-}
+}
\ No newline at end of file

Propchange: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/DefaultComponentReferencePropertiesTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/DefaultComponentReferencePropertiesTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Propchange: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/impl/DefaultComponentReferencePropertiesTest.java
------------------------------------------------------------------------------
    svn:mergeinfo = 



Re: svn commit: r724319 - in /activemq/camel/trunk/camel-core/src: main/java/org/apache/camel/component/file/ main/java/org/apache/camel/impl/ test/java/org/apache/camel/component/file/ test/java/org/apache/camel/impl/

Posted by Claus Ibsen <cl...@gmail.com>.
Hi Willem

Spot on. Thanks for the great review.


/Claus Ibsen
Apache Camel Committer
Blog: http://davsclaus.blogspot.com/



On Tue, Dec 9, 2008 at 6:44 AM, Willem Jiang <wi...@gmail.com> wrote:
> Hi Claus,
>
> No, if you take a look at HttpComponent and CxfSoapComponent, you will
> find these component still call the setProperties method, even the
> useIntrospectionOnEndpoint = false.
>
> They have their own parameter interceptions when they create the
> endpoint, such as
>
> Map soapProps = IntrospectionSupport.extractProperties(parameters, "soap.");
>
> HttpClientParams params = new HttpClientParams();
> IntrospectionSupport.setProperties(params, parameters, "httpClient.");
>
> I think they just don't want the URI parameters has the side effect on
> the endpoint.
>
>
> Willem
>
> Claus Ibsen wrote:
>> Hi William
>>
>> Isn't it the idea of the useIntrospectionOnEndpoint=false to not allow
>> Camel to do this auto discovery of setters?
>> So to be safe I added this check as well.
>>
>> BTW: Which components set this to false?
>>
>> /Claus Ibsen
>> Apache Camel Committer
>> Blog: http://davsclaus.blogspot.com/
>>
>>
>>
>> On Tue, Dec 9, 2008 at 3:53 AM, Willem Jiang <wi...@gmail.com> wrote:
>>> Hi Claus,
>>>
>>> The change of the DefaultComponent setProperties() will be blocked to
>>> the components which extend the DefaultComponent and set the
>>> useIntrospectionOnEndpoint to be false.
>>>
>>> So I will remove the check of useIntrospectionOnEndpoint() in the
>>> setProperties in my next commit to fixed the unit test error.
>>>
>>> Willem
>>>
>>>
>>> davsclaus@apache.org wrote:
>>>> Author: davsclaus
>>>> Date: Mon Dec  8 03:55:04 2008
>>>> New Revision: 724319
>>>>
>>>> URL: http://svn.apache.org/viewvc?rev=724319&view=rev
>>>> Log:
>>>> CAMEL-895: Added support for # notation in endpoint URIs to lookup a bean in the registry
>>>>
>>> ......
>>>> Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java
>>>> URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java?rev=724319&r1=724318&r2=724319&view=diff
>>>> ==============================================================================
>>>> --- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java (original)
>>>> +++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java Mon Dec  8 03:55:04 2008
>>>> @@ -17,6 +17,7 @@
>>>>  package org.apache.camel.impl;
>>>> @@ -185,7 +186,48 @@
>>>>       * Sets the bean properties on the given bean
>>>>       */
>>>>      protected void setProperties(Object bean, Map parameters) throws Exception {
>>>> -        IntrospectionSupport.setProperties(getCamelContext().getTypeConverter(), bean, parameters);
>>>> +        if (useIntrospectionOnEndpoint()) {
>>>> +            // set reference properties first as they use # syntax that fools the regular properties setter
>>>> +            setReferenceProperties(bean, parameters);
>>>> +            IntrospectionSupport.setProperties(getCamelContext().getTypeConverter(), bean, parameters);
>>>> +        }
>>>> +    }
>>
>
>

Re: svn commit: r724319 - in /activemq/camel/trunk/camel-core/src: main/java/org/apache/camel/component/file/ main/java/org/apache/camel/impl/ test/java/org/apache/camel/component/file/ test/java/org/apache/camel/impl/

Posted by Willem Jiang <wi...@gmail.com>.
Hi Claus,

No, if you take a look at HttpComponent and CxfSoapComponent, you will
find these component still call the setProperties method, even the
useIntrospectionOnEndpoint = false.

They have their own parameter interceptions when they create the
endpoint, such as

Map soapProps = IntrospectionSupport.extractProperties(parameters, "soap.");

HttpClientParams params = new HttpClientParams();
IntrospectionSupport.setProperties(params, parameters, "httpClient.");

I think they just don't want the URI parameters has the side effect on
the endpoint.


Willem

Claus Ibsen wrote:
> Hi William
> 
> Isn't it the idea of the useIntrospectionOnEndpoint=false to not allow
> Camel to do this auto discovery of setters?
> So to be safe I added this check as well.
> 
> BTW: Which components set this to false?
> 
> /Claus Ibsen
> Apache Camel Committer
> Blog: http://davsclaus.blogspot.com/
> 
> 
> 
> On Tue, Dec 9, 2008 at 3:53 AM, Willem Jiang <wi...@gmail.com> wrote:
>> Hi Claus,
>>
>> The change of the DefaultComponent setProperties() will be blocked to
>> the components which extend the DefaultComponent and set the
>> useIntrospectionOnEndpoint to be false.
>>
>> So I will remove the check of useIntrospectionOnEndpoint() in the
>> setProperties in my next commit to fixed the unit test error.
>>
>> Willem
>>
>>
>> davsclaus@apache.org wrote:
>>> Author: davsclaus
>>> Date: Mon Dec  8 03:55:04 2008
>>> New Revision: 724319
>>>
>>> URL: http://svn.apache.org/viewvc?rev=724319&view=rev
>>> Log:
>>> CAMEL-895: Added support for # notation in endpoint URIs to lookup a bean in the registry
>>>
>> ......
>>> Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java
>>> URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java?rev=724319&r1=724318&r2=724319&view=diff
>>> ==============================================================================
>>> --- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java (original)
>>> +++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java Mon Dec  8 03:55:04 2008
>>> @@ -17,6 +17,7 @@
>>>  package org.apache.camel.impl;
>>> @@ -185,7 +186,48 @@
>>>       * Sets the bean properties on the given bean
>>>       */
>>>      protected void setProperties(Object bean, Map parameters) throws Exception {
>>> -        IntrospectionSupport.setProperties(getCamelContext().getTypeConverter(), bean, parameters);
>>> +        if (useIntrospectionOnEndpoint()) {
>>> +            // set reference properties first as they use # syntax that fools the regular properties setter
>>> +            setReferenceProperties(bean, parameters);
>>> +            IntrospectionSupport.setProperties(getCamelContext().getTypeConverter(), bean, parameters);
>>> +        }
>>> +    }
> 


Re: svn commit: r724319 - in /activemq/camel/trunk/camel-core/src: main/java/org/apache/camel/component/file/ main/java/org/apache/camel/impl/ test/java/org/apache/camel/component/file/ test/java/org/apache/camel/impl/

Posted by Claus Ibsen <cl...@gmail.com>.
Hi

Sorry got your named wrong - Willem.

/Claus Ibsen
Apache Camel Committer
Blog: http://davsclaus.blogspot.com/



On Tue, Dec 9, 2008 at 5:57 AM, Claus Ibsen <cl...@gmail.com> wrote:
> Hi William
>
> Isn't it the idea of the useIntrospectionOnEndpoint=false to not allow
> Camel to do this auto discovery of setters?
> So to be safe I added this check as well.
>
> BTW: Which components set this to false?
>
> /Claus Ibsen
> Apache Camel Committer
> Blog: http://davsclaus.blogspot.com/
>
>
>
> On Tue, Dec 9, 2008 at 3:53 AM, Willem Jiang <wi...@gmail.com> wrote:
>> Hi Claus,
>>
>> The change of the DefaultComponent setProperties() will be blocked to
>> the components which extend the DefaultComponent and set the
>> useIntrospectionOnEndpoint to be false.
>>
>> So I will remove the check of useIntrospectionOnEndpoint() in the
>> setProperties in my next commit to fixed the unit test error.
>>
>> Willem
>>
>>
>> davsclaus@apache.org wrote:
>>> Author: davsclaus
>>> Date: Mon Dec  8 03:55:04 2008
>>> New Revision: 724319
>>>
>>> URL: http://svn.apache.org/viewvc?rev=724319&view=rev
>>> Log:
>>> CAMEL-895: Added support for # notation in endpoint URIs to lookup a bean in the registry
>>>
>> ......
>>>
>>> Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java
>>> URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java?rev=724319&r1=724318&r2=724319&view=diff
>>> ==============================================================================
>>> --- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java (original)
>>> +++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java Mon Dec  8 03:55:04 2008
>>> @@ -17,6 +17,7 @@
>>>  package org.apache.camel.impl;
>>
>>> @@ -185,7 +186,48 @@
>>>       * Sets the bean properties on the given bean
>>>       */
>>>      protected void setProperties(Object bean, Map parameters) throws Exception {
>>> -        IntrospectionSupport.setProperties(getCamelContext().getTypeConverter(), bean, parameters);
>>> +        if (useIntrospectionOnEndpoint()) {
>>> +            // set reference properties first as they use # syntax that fools the regular properties setter
>>> +            setReferenceProperties(bean, parameters);
>>> +            IntrospectionSupport.setProperties(getCamelContext().getTypeConverter(), bean, parameters);
>>> +        }
>>> +    }
>>
>

Re: svn commit: r724319 - in /activemq/camel/trunk/camel-core/src: main/java/org/apache/camel/component/file/ main/java/org/apache/camel/impl/ test/java/org/apache/camel/component/file/ test/java/org/apache/camel/impl/

Posted by Claus Ibsen <cl...@gmail.com>.
Hi William

Isn't it the idea of the useIntrospectionOnEndpoint=false to not allow
Camel to do this auto discovery of setters?
So to be safe I added this check as well.

BTW: Which components set this to false?

/Claus Ibsen
Apache Camel Committer
Blog: http://davsclaus.blogspot.com/



On Tue, Dec 9, 2008 at 3:53 AM, Willem Jiang <wi...@gmail.com> wrote:
> Hi Claus,
>
> The change of the DefaultComponent setProperties() will be blocked to
> the components which extend the DefaultComponent and set the
> useIntrospectionOnEndpoint to be false.
>
> So I will remove the check of useIntrospectionOnEndpoint() in the
> setProperties in my next commit to fixed the unit test error.
>
> Willem
>
>
> davsclaus@apache.org wrote:
>> Author: davsclaus
>> Date: Mon Dec  8 03:55:04 2008
>> New Revision: 724319
>>
>> URL: http://svn.apache.org/viewvc?rev=724319&view=rev
>> Log:
>> CAMEL-895: Added support for # notation in endpoint URIs to lookup a bean in the registry
>>
> ......
>>
>> Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java
>> URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java?rev=724319&r1=724318&r2=724319&view=diff
>> ==============================================================================
>> --- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java (original)
>> +++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java Mon Dec  8 03:55:04 2008
>> @@ -17,6 +17,7 @@
>>  package org.apache.camel.impl;
>
>> @@ -185,7 +186,48 @@
>>       * Sets the bean properties on the given bean
>>       */
>>      protected void setProperties(Object bean, Map parameters) throws Exception {
>> -        IntrospectionSupport.setProperties(getCamelContext().getTypeConverter(), bean, parameters);
>> +        if (useIntrospectionOnEndpoint()) {
>> +            // set reference properties first as they use # syntax that fools the regular properties setter
>> +            setReferenceProperties(bean, parameters);
>> +            IntrospectionSupport.setProperties(getCamelContext().getTypeConverter(), bean, parameters);
>> +        }
>> +    }
>

Re: svn commit: r724319 - in /activemq/camel/trunk/camel-core/src: main/java/org/apache/camel/component/file/ main/java/org/apache/camel/impl/ test/java/org/apache/camel/component/file/ test/java/org/apache/camel/impl/

Posted by Willem Jiang <wi...@gmail.com>.
Hi Claus,

The change of the DefaultComponent setProperties() will be blocked to
the components which extend the DefaultComponent and set the
useIntrospectionOnEndpoint to be false.

So I will remove the check of useIntrospectionOnEndpoint() in the
setProperties in my next commit to fixed the unit test error.

Willem


davsclaus@apache.org wrote:
> Author: davsclaus
> Date: Mon Dec  8 03:55:04 2008
> New Revision: 724319
> 
> URL: http://svn.apache.org/viewvc?rev=724319&view=rev
> Log:
> CAMEL-895: Added support for # notation in endpoint URIs to lookup a bean in the registry
> 
......
> 
> Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java
> URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java?rev=724319&r1=724318&r2=724319&view=diff
> ==============================================================================
> --- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java (original)
> +++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultComponent.java Mon Dec  8 03:55:04 2008
> @@ -17,6 +17,7 @@
>  package org.apache.camel.impl;

> @@ -185,7 +186,48 @@
>       * Sets the bean properties on the given bean
>       */
>      protected void setProperties(Object bean, Map parameters) throws Exception {
> -        IntrospectionSupport.setProperties(getCamelContext().getTypeConverter(), bean, parameters);
> +        if (useIntrospectionOnEndpoint()) {
> +            // set reference properties first as they use # syntax that fools the regular properties setter
> +            setReferenceProperties(bean, parameters);
> +            IntrospectionSupport.setProperties(getCamelContext().getTypeConverter(), bean, parameters);
> +        }
> +    }