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 2020/03/13 15:52:05 UTC

[camel] 01/04: CAMEL-14670: @BeanConfigInject for injecting bean configuration classes that are pre configured from properties files. Also add support for using configurer classes with option prefix in PropertyBindingSupport.

This is an automated email from the ASF dual-hosted git repository.

davsclaus pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel.git

commit db53fd669a40f27dbd05ac6414f00868f57f7fd2
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Fri Mar 13 14:11:45 2020 +0100

    CAMEL-14670: @BeanConfigInject for injecting bean configuration classes that are pre configured from properties files. Also add support for using configurer classes with option prefix in PropertyBindingSupport.
---
 .../java/org/apache/camel/BeanConfigInject.java    |  39 +++++++
 .../impl/engine/DefaultCamelBeanPostProcessor.java |  70 ++++++++++++
 .../MainIoCBeanConfigInjectConfigurerTest.java     | 122 +++++++++++++++++++++
 ...rTest.java => MainIoCBeanConfigInjectTest.java} |  65 +++++------
 .../camel/main/MainIoCNewRouteBuilderTest.java     |   2 +-
 .../camel/support/PropertyBindingSupport.java      |  11 +-
 6 files changed, 271 insertions(+), 38 deletions(-)

diff --git a/core/camel-api/src/main/java/org/apache/camel/BeanConfigInject.java b/core/camel-api/src/main/java/org/apache/camel/BeanConfigInject.java
new file mode 100644
index 0000000..edb120b
--- /dev/null
+++ b/core/camel-api/src/main/java/org/apache/camel/BeanConfigInject.java
@@ -0,0 +1,39 @@
+/*
+ * 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;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Used to indicate an injection point of a configuration bean (obtained from the {@link org.apache.camel.spi.Registry},
+ * or a new instance is created) into a POJO.
+ */
+@Retention(RetentionPolicy.RUNTIME)
+@Documented
+@Target({ElementType.FIELD, ElementType.METHOD, ElementType.CONSTRUCTOR, ElementType.PARAMETER})
+public @interface BeanConfigInject {
+
+    /**
+     * Name of the root property (prefix)
+     */
+    String value();
+
+}
diff --git a/core/camel-base/src/main/java/org/apache/camel/impl/engine/DefaultCamelBeanPostProcessor.java b/core/camel-base/src/main/java/org/apache/camel/impl/engine/DefaultCamelBeanPostProcessor.java
index d1882c9..ae90f08 100644
--- a/core/camel-base/src/main/java/org/apache/camel/impl/engine/DefaultCamelBeanPostProcessor.java
+++ b/core/camel-base/src/main/java/org/apache/camel/impl/engine/DefaultCamelBeanPostProcessor.java
@@ -21,13 +21,17 @@ import java.lang.reflect.Field;
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Comparator;
+import java.util.LinkedHashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Properties;
 import java.util.Set;
 
 import org.apache.camel.BeanInject;
 import org.apache.camel.BindToRegistry;
 import org.apache.camel.CamelContext;
 import org.apache.camel.CamelContextAware;
+import org.apache.camel.BeanConfigInject;
 import org.apache.camel.DeferredContextBinding;
 import org.apache.camel.EndpointInject;
 import org.apache.camel.ExtendedCamelContext;
@@ -37,15 +41,18 @@ import org.apache.camel.PropertyInject;
 import org.apache.camel.RuntimeCamelException;
 import org.apache.camel.TypeConverter;
 import org.apache.camel.spi.CamelBeanPostProcessor;
+import org.apache.camel.spi.GeneratedPropertyConfigurer;
 import org.apache.camel.spi.PropertiesComponent;
 import org.apache.camel.spi.Registry;
 import org.apache.camel.support.DefaultEndpoint;
+import org.apache.camel.support.PropertyBindingSupport;
 import org.apache.camel.util.ReflectionHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import static org.apache.camel.support.ObjectHelper.invokeMethod;
 import static org.apache.camel.util.ObjectHelper.isEmpty;
+import static org.apache.camel.util.ObjectHelper.loadClass;
 
 /**
  * A bean post processor which implements the <a href="http://camel.apache.org/bean-integration.html">Bean Integration</a>
@@ -475,6 +482,14 @@ public class DefaultCamelBeanPostProcessor implements CamelBeanPostProcessor {
                         } catch (Exception e) {
                             throw RuntimeCamelException.wrapRuntimeCamelException(e);
                         }
+                    } else if (ann.annotationType() == BeanConfigInject.class) {
+                        BeanConfigInject pi = (BeanConfigInject) ann;
+                        // build key with default value included as this is supported during resolving
+                        // it may be a configuration class which we want to instantiate and configure with
+                        // project inject as base keys
+                        ExtendedCamelContext ecc = (ExtendedCamelContext) getOrLookupCamelContext();
+                        Object result = resolveBeanConfigInject(ecc, pi, type);
+                        parameters[i] = result;
                     } else if (ann.annotationType() == BeanInject.class) {
                         BeanInject bi = (BeanInject) ann;
                         String key = bi.value();
@@ -522,4 +537,59 @@ public class DefaultCamelBeanPostProcessor implements CamelBeanPostProcessor {
         return parameters;
     }
 
+    private Object resolveBeanConfigInject(ExtendedCamelContext ecc, BeanConfigInject pi, Class<?> type) {
+        // create an instance of type
+        Object bean;
+        Set<?> instances = ecc.getRegistry().findByType(type);
+        if (instances.size() == 1) {
+            bean = instances.iterator().next();
+        } else if (instances.size() > 1) {
+            return null;
+        } else {
+            // attempt to create a new instance
+            try {
+                bean = ecc.getInjector().newInstance(type);
+            } catch (Throwable e) {
+                // ignore
+                return null;
+            }
+        }
+
+        // root key
+        String rootKey = pi.value();
+        // clip trailing dot
+        if (rootKey.endsWith(".")) {
+            rootKey = rootKey.substring(0, rootKey.length() - 1);
+        }
+
+        // get all properties and transfer to map
+        Properties props = ecc.getPropertiesComponent().loadProperties();
+        Map<String, Object> map = new LinkedHashMap<>();
+        for (String key : props.stringPropertyNames()) {
+            map.put(key, props.getProperty(key));
+        }
+
+        // lookup configurer if there is any
+        // use FQN class name first, then simple name, and root key last
+        GeneratedPropertyConfigurer configurer = null;
+        String[] names = new String[]{type.getName() + "-configurer", type.getSimpleName() + "-configurer", rootKey + "-configurer"};
+        for (String name : names) {
+            configurer = ecc.getConfigurerResolver().resolvePropertyConfigurer(name, ecc);
+            if (configurer != null) {
+                break;
+            }
+        }
+
+        new PropertyBindingSupport.Builder()
+            .withCamelContext(ecc)
+            .withIgnoreCase(true)
+            .withTarget(bean)
+            .withConfigurer(configurer)
+            .withOptionPrefix(rootKey + ".")
+            .withProperties(map)
+            .bind();
+
+        return bean;
+    }
+
 }
diff --git a/core/camel-main/src/test/java/org/apache/camel/main/MainIoCBeanConfigInjectConfigurerTest.java b/core/camel-main/src/test/java/org/apache/camel/main/MainIoCBeanConfigInjectConfigurerTest.java
new file mode 100644
index 0000000..6b857cd
--- /dev/null
+++ b/core/camel-main/src/test/java/org/apache/camel/main/MainIoCBeanConfigInjectConfigurerTest.java
@@ -0,0 +1,122 @@
+/*
+ * 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.main;
+
+import org.apache.camel.BeanConfigInject;
+import org.apache.camel.BindToRegistry;
+import org.apache.camel.CamelContext;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.spi.GeneratedPropertyConfigurer;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class MainIoCBeanConfigInjectConfigurerTest extends Assert {
+
+    @Test
+    public void testMainIoC() throws Exception {
+        Main main = new Main();
+        main.addRoutesBuilder(new MyRouteBuilder());
+        main.addInitialProperty("bar.name", "Thirsty Bear");
+        main.addInitialProperty("bar.age", "23");
+        main.bind(MyBarConfig.class.getName() + "-configurer", new MyBarConfigConfigurer());
+        main.start();
+
+        CamelContext camelContext = main.getCamelContext();
+        assertNotNull(camelContext);
+
+        MockEndpoint endpoint = camelContext.getEndpoint("mock:results", MockEndpoint.class);
+        endpoint.expectedBodiesReceived("The Thirsty Bear (minimum age: 46)");
+
+        main.getCamelTemplate().sendBody("direct:start", "Which bar");
+
+        endpoint.assertIsSatisfied();
+
+        main.stop();
+    }
+
+    public static class MyBar {
+
+        private final String description;
+
+        public MyBar(String description) {
+            this.description = description;
+        }
+
+        @Override
+        public String toString() {
+            return description;
+        }
+    }
+
+    public static class MyBarConfig {
+
+        private String name;
+        private int age;
+
+        public String getName() {
+            return name;
+        }
+
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        public int getAge() {
+            return age;
+        }
+
+        public void setAge(int age) {
+            this.age = age;
+        }
+    }
+
+    public static class MyBarConfigConfigurer implements GeneratedPropertyConfigurer {
+
+        @Override
+        public boolean configure(CamelContext camelContext, Object target, String name, Object value, boolean ignoreCase) {
+            if (target instanceof MyBarConfig) {
+                MyBarConfig config = (MyBarConfig) target;
+                if ("name".equals(name)) {
+                    // ensure the configurer was in use by prefix
+                    config.setName("The " + value.toString());
+                    return true;
+                } else if ("age".equals(name)) {
+                    // ensure the configurer was in use by * 2
+                    int num = Integer.parseInt(value.toString()) * 2;
+                    config.setAge(num);
+                    return true;
+                }
+            }
+            return false;
+        }
+    }
+
+    public static class MyRouteBuilder extends RouteBuilder {
+
+        @BindToRegistry("bar")
+        public MyBar createBar(@BeanConfigInject("bar") MyBarConfig config) {
+            String text = config.getName() + " (minimum age: " + config.getAge() + ")";
+            return new MyBar(text);
+        }
+
+        @Override
+        public void configure() throws Exception {
+            from("direct:start").bean("bar").to("mock:results");
+        }
+    }
+}
diff --git a/core/camel-main/src/test/java/org/apache/camel/main/MainIoCNewRouteBuilderTest.java b/core/camel-main/src/test/java/org/apache/camel/main/MainIoCBeanConfigInjectTest.java
similarity index 61%
copy from core/camel-main/src/test/java/org/apache/camel/main/MainIoCNewRouteBuilderTest.java
copy to core/camel-main/src/test/java/org/apache/camel/main/MainIoCBeanConfigInjectTest.java
index d41bc2c..2dae49d 100644
--- a/core/camel-main/src/test/java/org/apache/camel/main/MainIoCNewRouteBuilderTest.java
+++ b/core/camel-main/src/test/java/org/apache/camel/main/MainIoCBeanConfigInjectTest.java
@@ -16,84 +16,79 @@
  */
 package org.apache.camel.main;
 
-import org.apache.camel.BeanInject;
 import org.apache.camel.BindToRegistry;
 import org.apache.camel.CamelContext;
-import org.apache.camel.PropertyInject;
+import org.apache.camel.BeanConfigInject;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.component.mock.MockEndpoint;
 import org.junit.Assert;
 import org.junit.Test;
 
-public class MainIoCNewRouteBuilderTest extends Assert {
+public class MainIoCBeanConfigInjectTest extends Assert {
 
     @Test
     public void testMainIoC() throws Exception {
         Main main = new Main();
-        main.addConfiguration(new MyConfiguration());
         main.addRoutesBuilder(new MyRouteBuilder());
+        main.addInitialProperty("bar.name", "Thirsty Bear");
+        main.addInitialProperty("bar.age", "23");
         main.start();
 
         CamelContext camelContext = main.getCamelContext();
         assertNotNull(camelContext);
 
         MockEndpoint endpoint = camelContext.getEndpoint("mock:results", MockEndpoint.class);
-        endpoint.expectedBodiesReceived("World");
+        endpoint.expectedBodiesReceived("Thirsty Bear (minimum age: 23)");
 
-        main.getCamelTemplate().sendBody("direct:start", "<message>1</message>");
+        main.getCamelTemplate().sendBody("direct:start", "Which bar");
 
         endpoint.assertIsSatisfied();
 
-        MainIoCNewRouteBuilderTest.MyConfiguration.MyCoolBean mcb = (MainIoCNewRouteBuilderTest.MyConfiguration.MyCoolBean) camelContext.getRegistry().lookupByName("MyCoolBean");
-        assertNotNull(mcb);
-        assertEquals("Tiger", mcb.getName());
-
         main.stop();
     }
 
-    public static class MyConfiguration {
-
-        @BeanInject
-        private CamelContext camel;
-
-        @BindToRegistry
-        public static class MyCoolBean {
-
-            private String name = "Tiger";
+    public static class MyBar {
 
-            public String getName() {
-                return name;
-            }
+        private final String description;
 
-            public void setName(String name) {
-                this.name = name;
-            }
+        public MyBar(String description) {
+            this.description = description;
         }
 
-        public void configure() {
-            camel.getGlobalOptions().put("foo", "123");
+        @Override
+        public String toString() {
+            return description;
         }
     }
 
-    public static class MyBar {
+    public static class MyBarConfig {
 
-        private final String name;
+        private String name;
+        private int age;
 
-        public MyBar(String name) {
+        public String getName() {
+            return name;
+        }
+
+        public void setName(String name) {
             this.name = name;
         }
 
-        @Override
-        public String toString() {
-            return name;
+        public int getAge() {
+            return age;
+        }
+
+        public void setAge(int age) {
+            this.age = age;
         }
     }
 
     public static class MyRouteBuilder extends RouteBuilder {
 
         @BindToRegistry("bar")
-        public MyBar createBar(@PropertyInject(value = "hello") String hello) {
-            return new MyBar(hello);
+        public MyBar createBar(@BeanConfigInject("bar") MyBarConfig config) {
+            String text = config.getName() + " (minimum age: " + config.getAge() + ")";
+            return new MyBar(text);
         }
 
         @Override
diff --git a/core/camel-main/src/test/java/org/apache/camel/main/MainIoCNewRouteBuilderTest.java b/core/camel-main/src/test/java/org/apache/camel/main/MainIoCNewRouteBuilderTest.java
index d41bc2c..ca0d7da 100644
--- a/core/camel-main/src/test/java/org/apache/camel/main/MainIoCNewRouteBuilderTest.java
+++ b/core/camel-main/src/test/java/org/apache/camel/main/MainIoCNewRouteBuilderTest.java
@@ -92,7 +92,7 @@ public class MainIoCNewRouteBuilderTest extends Assert {
     public static class MyRouteBuilder extends RouteBuilder {
 
         @BindToRegistry("bar")
-        public MyBar createBar(@PropertyInject(value = "hello") String hello) {
+        public MyBar createBar(@PropertyInject("hello") String hello) {
             return new MyBar(hello);
         }
 
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/PropertyBindingSupport.java b/core/camel-support/src/main/java/org/apache/camel/support/PropertyBindingSupport.java
index 2ca1b5d..9ed54cc 100644
--- a/core/camel-support/src/main/java/org/apache/camel/support/PropertyBindingSupport.java
+++ b/core/camel-support/src/main/java/org/apache/camel/support/PropertyBindingSupport.java
@@ -197,7 +197,7 @@ public final class PropertyBindingSupport {
         }
 
         /**
-         * Whether properties should be filtered by prefix.         *
+         * Whether properties should be filtered by prefix.
          * Note that the prefix is removed from the key before the property is bound.
          */
         public Builder withOptionPrefix(String optionPrefix) {
@@ -509,7 +509,14 @@ public final class PropertyBindingSupport {
                     key = key.substring(optionPrefix.length());
                 }
 
-                boolean bound = bindProperty(camelContext, target, key, value, ignoreCase, nesting, deepNesting, fluentBuilder, allowPrivateSetter, reference, placeholder);
+                boolean bound = false;
+                if (configurer != null) {
+                    // attempt configurer first
+                    bound = configurer.configure(camelContext, target, key, value, ignoreCase);
+                }
+                if (!bound) {
+                    bound = bindProperty(camelContext, target, key, value, ignoreCase, nesting, deepNesting, fluentBuilder, allowPrivateSetter, reference, placeholder);
+                }
                 if (bound && removeParameter) {
                     properties.remove(propertyKey);
                 }