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 2019/08/20 19:05:22 UTC

[camel] 02/12: CAMEL-13870: Fast property configuration of Camel endpoints. Work in progress.

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

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

commit 1d4c0977110ddffe1ec6d41b21b2eed10a8a6b24
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Tue Aug 20 09:16:24 2019 +0200

    CAMEL-13870: Fast property configuration of Camel endpoints. Work in progress.
---
 .../apache/camel/component/log/LogEndpoint.java    |   6 ++
 .../camel/component/log/LogEndpointConfigurer.java |  50 +++++++++
 .../apache/camel/spi/PropertyConfigurerAware.java  |  23 ++++
 .../camel/processor/SimpleMockPlaceholderTest.java |   2 +-
 .../org/apache/camel/support/DefaultComponent.java |   8 +-
 .../org/apache/camel/support/DefaultEndpoint.java  |  15 ++-
 .../camel/support/PropertyBindingSupport.java      |  51 +++++++--
 .../apache/camel/itest/jmh/LogEndpointTest.java    | 102 ++++++++++++++++++
 .../camel/itest/jmh/SimpleMockPlaceholderTest.java | 117 +++++++++++++++++++++
 .../camel-jmh/src/test/resources/log4j2.properties |   2 +-
 10 files changed, 363 insertions(+), 13 deletions(-)

diff --git a/components/camel-log/src/main/java/org/apache/camel/component/log/LogEndpoint.java b/components/camel-log/src/main/java/org/apache/camel/component/log/LogEndpoint.java
index 5c53d5d..f7119d7 100644
--- a/components/camel-log/src/main/java/org/apache/camel/component/log/LogEndpoint.java
+++ b/components/camel-log/src/main/java/org/apache/camel/component/log/LogEndpoint.java
@@ -25,6 +25,7 @@ import org.apache.camel.spi.CamelLogger;
 import org.apache.camel.spi.ExchangeFormatter;
 import org.apache.camel.spi.MaskingFormatter;
 import org.apache.camel.spi.Metadata;
+import org.apache.camel.spi.PropertyConfigurer;
 import org.apache.camel.spi.UriEndpoint;
 import org.apache.camel.spi.UriParam;
 import org.apache.camel.spi.UriPath;
@@ -81,6 +82,11 @@ public class LogEndpoint extends ProcessorEndpoint {
     }
 
     @Override
+    public PropertyConfigurer getPropertyConfigurer() {
+        return new LogEndpointConfigurer(this, getCamelContext());
+    }
+
+    @Override
     protected void doStart() throws Exception {
         if (logger == null) {
             logger = createLogger();
diff --git a/components/camel-log/src/main/java/org/apache/camel/component/log/LogEndpointConfigurer.java b/components/camel-log/src/main/java/org/apache/camel/component/log/LogEndpointConfigurer.java
new file mode 100644
index 0000000..206d10c
--- /dev/null
+++ b/components/camel-log/src/main/java/org/apache/camel/component/log/LogEndpointConfigurer.java
@@ -0,0 +1,50 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.log;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.spi.PropertyConfigurer;
+
+public class LogEndpointConfigurer implements PropertyConfigurer<Object> {
+
+    private final Map<String, Supplier<Object>> readPlaceholders = new HashMap<>();
+    private final Map<String, Consumer<Object>> writePlaceholders = new HashMap<>();
+
+    public LogEndpointConfigurer(final Object target, final CamelContext camelContext) {
+        final LogEndpoint endpoint = (LogEndpoint) target;
+
+        readPlaceholders.put("loggerName", endpoint::getLoggerName);
+        writePlaceholders.put("loggerName", o -> endpoint.setLoggerName(camelContext.getTypeConverter().convertTo(String.class, o)));
+        readPlaceholders.put("groupSize", endpoint::getGroupSize);
+        writePlaceholders.put("groupSize", o -> endpoint.setGroupSize(camelContext.getTypeConverter().convertTo(Integer.class, o)));
+    }
+
+    @Override
+    public Map<String, Supplier<Object>> getReadPropertyPlaceholderOptions(CamelContext camelContext) {
+        return readPlaceholders;
+    }
+
+    @Override
+    public Map<String, Consumer<Object>> getWritePropertyPlaceholderOptions(CamelContext camelContext) {
+        return writePlaceholders;
+    }
+}
diff --git a/core/camel-api/src/main/java/org/apache/camel/spi/PropertyConfigurerAware.java b/core/camel-api/src/main/java/org/apache/camel/spi/PropertyConfigurerAware.java
new file mode 100644
index 0000000..7223e41
--- /dev/null
+++ b/core/camel-api/src/main/java/org/apache/camel/spi/PropertyConfigurerAware.java
@@ -0,0 +1,23 @@
+/*
+ * 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.spi;
+
+public interface PropertyConfigurerAware {
+
+    PropertyConfigurer getPropertyConfigurer();
+
+}
diff --git a/core/camel-core/src/test/java/org/apache/camel/processor/SimpleMockPlaceholderTest.java b/core/camel-core/src/test/java/org/apache/camel/processor/SimpleMockPlaceholderTest.java
index dd8b986..be92152 100644
--- a/core/camel-core/src/test/java/org/apache/camel/processor/SimpleMockPlaceholderTest.java
+++ b/core/camel-core/src/test/java/org/apache/camel/processor/SimpleMockPlaceholderTest.java
@@ -68,7 +68,7 @@ public class SimpleMockPlaceholderTest extends ContextTestSupport {
         return new RouteBuilder() {
             @Override
             public void configure() throws Exception {
-                from("direct:start").to("{{foo}}").to("log:bar").to("mock:{{end}}");
+                from("direct:start?block=true").to("{{foo}}").to("log:bar?groupSize=3").to("mock:{{end}}");
             }
         };
     }
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/DefaultComponent.java b/core/camel-support/src/main/java/org/apache/camel/support/DefaultComponent.java
index 7d4b73d..591260d 100644
--- a/core/camel-support/src/main/java/org/apache/camel/support/DefaultComponent.java
+++ b/core/camel-support/src/main/java/org/apache/camel/support/DefaultComponent.java
@@ -34,6 +34,8 @@ import org.apache.camel.Endpoint;
 import org.apache.camel.ResolveEndpointFailedException;
 import org.apache.camel.component.extension.ComponentExtension;
 import org.apache.camel.spi.Metadata;
+import org.apache.camel.spi.PropertyConfigurer;
+import org.apache.camel.spi.PropertyConfigurerAware;
 import org.apache.camel.support.service.ServiceSupport;
 import org.apache.camel.util.ObjectHelper;
 import org.apache.camel.util.URISupport;
@@ -366,8 +368,12 @@ public abstract class DefaultComponent extends ServiceSupport implements Compone
                     .withPlaceholder(false).withNesting(false).withDeepNesting(false).withReference(false)
                     .bind(camelContext, bean, parameters);
         } else {
+            PropertyConfigurer configurer = null;
+            if (bean instanceof PropertyConfigurerAware) {
+                configurer = ((PropertyConfigurerAware) bean).getPropertyConfigurer();
+            }
             // use advanced binding
-            PropertyBindingSupport.build().bind(camelContext, bean, parameters);
+            PropertyBindingSupport.build().withConfigurer(configurer).bind(camelContext, bean, parameters);
         }
     }
 
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/DefaultEndpoint.java b/core/camel-support/src/main/java/org/apache/camel/support/DefaultEndpoint.java
index a57354f..d4d502f 100644
--- a/core/camel-support/src/main/java/org/apache/camel/support/DefaultEndpoint.java
+++ b/core/camel-support/src/main/java/org/apache/camel/support/DefaultEndpoint.java
@@ -32,6 +32,8 @@ import org.apache.camel.Producer;
 import org.apache.camel.ResolveEndpointFailedException;
 import org.apache.camel.spi.ExceptionHandler;
 import org.apache.camel.spi.HasId;
+import org.apache.camel.spi.PropertyConfigurer;
+import org.apache.camel.spi.PropertyConfigurerAware;
 import org.apache.camel.spi.UriParam;
 import org.apache.camel.support.service.ServiceSupport;
 import org.apache.camel.util.ObjectHelper;
@@ -49,7 +51,7 @@ import org.apache.camel.util.URISupport;
  * model or not. The option is default <tt>false</tt> which means asynchronous
  * processing is allowed.
  */
-public abstract class DefaultEndpoint extends ServiceSupport implements Endpoint, HasId, CamelContextAware {
+public abstract class DefaultEndpoint extends ServiceSupport implements Endpoint, HasId, CamelContextAware, PropertyConfigurerAware {
 
     private final String id = EndpointHelper.createEndpointId();
     private transient String endpointUriToString;
@@ -415,11 +417,20 @@ public abstract class DefaultEndpoint extends ServiceSupport implements Endpoint
                     .withPlaceholder(false).withNesting(false).withDeepNesting(false).withReference(false)
                     .bind(camelContext, bean, parameters);
         } else {
+            PropertyConfigurer configurer = null;
+            if (bean instanceof PropertyConfigurerAware) {
+                configurer = ((PropertyConfigurerAware) bean).getPropertyConfigurer();
+            }
             // use advanced binding
-            PropertyBindingSupport.build().bind(camelContext, bean, parameters);
+            PropertyBindingSupport.build().withConfigurer(configurer).bind(camelContext, bean, parameters);
         }
     }
 
+    @Override
+    public PropertyConfigurer getPropertyConfigurer() {
+        return null;
+    }
+
     /**
      * A factory method to lazily create the endpointUri if none is specified
      */
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 9060837..11ffbcc 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
@@ -25,9 +25,12 @@ import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 import java.util.Set;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.PropertyBindingException;
+import org.apache.camel.spi.PropertyConfigurer;
 import org.apache.camel.util.StringHelper;
 
 import static org.apache.camel.support.IntrospectionSupport.findSetterMethods;
@@ -73,6 +76,7 @@ public final class PropertyBindingSupport {
         private boolean allowPrivateSetter = true;
         private boolean ignoreCase;
         private String optionPrefix;
+        private PropertyConfigurer configurer;
 
         /**
          * CamelContext to be used
@@ -200,6 +204,14 @@ public final class PropertyBindingSupport {
         }
 
         /**
+         * Whether to use the configurer to configure the properties.
+         */
+        public Builder withConfigurer(PropertyConfigurer configurer) {
+            this.configurer = configurer;
+            return this;
+        }
+
+        /**
          * Binds the properties to the target object, and removes the property that was bound from properties.
          *
          * @return  true if one or more properties was bound
@@ -211,7 +223,7 @@ public final class PropertyBindingSupport {
             org.apache.camel.util.ObjectHelper.notNull(properties, "properties");
 
             return doBindProperties(camelContext, target, properties, optionPrefix, ignoreCase, removeParameters, mandatory,
-                    nesting, deepNesting, fluentBuilder, allowPrivateSetter, reference, placeholder);
+                    nesting, deepNesting, fluentBuilder, allowPrivateSetter, reference, placeholder, configurer);
         }
 
         /**
@@ -233,7 +245,7 @@ public final class PropertyBindingSupport {
             org.apache.camel.util.ObjectHelper.notNull(prop, "properties");
 
             return doBindProperties(context, obj, prop, optionPrefix, ignoreCase, removeParameters, mandatory,
-                    nesting, deepNesting, fluentBuilder, allowPrivateSetter, reference, placeholder);
+                    nesting, deepNesting, fluentBuilder, allowPrivateSetter, reference, placeholder, configurer);
         }
 
         /**
@@ -255,7 +267,7 @@ public final class PropertyBindingSupport {
             properties.put(key, value);
 
             return doBindProperties(camelContext, target, properties, optionPrefix, ignoreCase, removeParameters, mandatory,
-                    nesting, deepNesting, fluentBuilder, allowPrivateSetter, reference, placeholder);
+                    nesting, deepNesting, fluentBuilder, allowPrivateSetter, reference, placeholder, configurer);
         }
 
     }
@@ -424,7 +436,7 @@ public final class PropertyBindingSupport {
      * @param optionPrefix        the prefix used to filter properties
      * @param ignoreCase          whether to ignore case for property keys
      * @param removeParameter     whether to remove bound parameters
-     * @param madatory            whether all parameters must be bound
+     * @param mandatory           whether all parameters must be bound
      * @param nesting             whether nesting is in use
      * @param deepNesting         whether deep nesting is in use, where Camel will attempt to walk as deep as possible by creating new objects in the OGNL graph if
      *                            a property has a setter and the object can be created from a default no-arg constructor.
@@ -432,21 +444,44 @@ public final class PropertyBindingSupport {
      * @param allowPrivateSetter  whether autowiring components allows to use private setter method when setting the value
      * @param reference           whether reference parameter (syntax starts with #) is in use
      * @param placeholder         whether to use Camels property placeholder to resolve placeholders on keys and values
+     * @param configurer          to use an optional {@link org.apache.camel.spi.PropertyConfigurer} to configure the properties
      * @return                    true if one or more properties was bound
      */
     private static boolean doBindProperties(CamelContext camelContext, Object target, Map<String, Object> properties,
-                                            String optionPrefix, boolean ignoreCase, boolean removeParameter, boolean madatory,
+                                            String optionPrefix, boolean ignoreCase, boolean removeParameter, boolean mandatory,
                                             boolean nesting, boolean deepNesting, boolean fluentBuilder, boolean allowPrivateSetter,
-                                            boolean reference, boolean placeholder) {
+                                            boolean reference, boolean placeholder,
+                                            PropertyConfigurer configurer) {
         org.apache.camel.util.ObjectHelper.notNull(camelContext, "camelContext");
         org.apache.camel.util.ObjectHelper.notNull(target, "target");
         org.apache.camel.util.ObjectHelper.notNull(properties, "properties");
         boolean rc = false;
 
+        // TODO: quick and dirty to only use configurer
+        if (configurer != null) {
+            Map<String, Consumer<Object>> writeProperties = configurer.getWritePropertyPlaceholderOptions(camelContext);
+            for (Iterator<Map.Entry<String, Object>> iter = properties.entrySet().iterator(); iter.hasNext();) {
+                Map.Entry<String, Object> entry = iter.next();
+                String key = entry.getKey();
+                Object value = entry.getValue();
+                if (writeProperties.containsKey(key)) {
+
+                    // TODO: reference
+                    // TODO: o
+
+                    writeProperties.get(key).accept(value);
+                    if (removeParameter) {
+                        properties.remove(key);
+                        rc = true;
+                    }
+                }
+            };
+        }
+
         // must set reference parameters first before the other bindings
         int size = properties.size();
         setReferenceProperties(camelContext, target, properties);
-        rc = properties.size() != size;
+        rc |= properties.size() != size;
 
         String uOptionPrefix = "";
         if (ignoreCase && isNotEmpty(optionPrefix)) {
@@ -471,7 +506,7 @@ public final class PropertyBindingSupport {
                 iter.remove();
                 rc = true;
             }
-            if (madatory && !bound) {
+            if (mandatory && !bound) {
                 throw new PropertyBindingException(target, key, value);
             }
         }
diff --git a/tests/camel-jmh/src/test/java/org/apache/camel/itest/jmh/LogEndpointTest.java b/tests/camel-jmh/src/test/java/org/apache/camel/itest/jmh/LogEndpointTest.java
new file mode 100644
index 0000000..eb2174f
--- /dev/null
+++ b/tests/camel-jmh/src/test/java/org/apache/camel/itest/jmh/LogEndpointTest.java
@@ -0,0 +1,102 @@
+/*
+ * 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.itest.jmh;
+
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Endpoint;
+import org.apache.camel.Exchange;
+import org.apache.camel.impl.DefaultCamelContext;
+import org.junit.Test;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.TearDown;
+import org.openjdk.jmh.infra.Blackhole;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+import org.openjdk.jmh.runner.options.TimeValue;
+
+/**
+ * Tests fast property binding on endpoints
+ */
+public class LogEndpointTest {
+
+    @Test
+    public void launchBenchmark() throws Exception {
+        Options opt = new OptionsBuilder()
+            // Specify which benchmarks to run.
+            // You can be more specific if you'd like to run only one benchmark per test.
+            .include(this.getClass().getName() + ".*")
+            // Set the following options as needed
+            .mode(Mode.All)
+            .timeUnit(TimeUnit.MICROSECONDS)
+            .warmupTime(TimeValue.seconds(1))
+            .warmupIterations(2)
+            .measurementTime(TimeValue.seconds(1))
+            .measurementIterations(2)
+            .threads(2)
+            .forks(1)
+            .shouldFailOnError(true)
+            .shouldDoGC(true)
+            .build();
+
+        new Runner(opt).run();
+    }
+
+    // The JMH samples are the best documentation for how to use it
+    // http://hg.openjdk.java.net/code-tools/jmh/file/tip/jmh-samples/src/main/java/org/openjdk/jmh/samples/
+    @State(Scope.Thread)
+    public static class BenchmarkState {
+        CamelContext camel;
+        AtomicInteger counter;
+
+        @Setup(Level.Trial)
+        public void initialize() {
+            camel = new DefaultCamelContext();
+            camel.getGlobalOptions().put(Exchange.MAXIMUM_ENDPOINT_CACHE_SIZE, "0");
+            counter = new AtomicInteger();
+        }
+
+        @TearDown(Level.Trial)
+        public void close() {
+            try {
+                camel.stop();
+            } catch (Exception e) {
+                // ignore
+            }
+        }
+
+    }
+
+    @Benchmark
+    @Measurement(batchSize = 1000)
+    public void logEndpoint(BenchmarkState state, Blackhole bh) {
+        // use the legacy binding which uses reflection
+        // Endpoint out = state.camel.getEndpoint("log:foo?basicPropertyBinding=true&groupSize=" + state.counter.incrementAndGet());
+        Endpoint out = state.camel.getEndpoint("log:foo?groupSize=" + state.counter.incrementAndGet());
+        bh.consume(out);
+    }
+
+}
diff --git a/tests/camel-jmh/src/test/java/org/apache/camel/itest/jmh/SimpleMockPlaceholderTest.java b/tests/camel-jmh/src/test/java/org/apache/camel/itest/jmh/SimpleMockPlaceholderTest.java
new file mode 100644
index 0000000..fb43d65
--- /dev/null
+++ b/tests/camel-jmh/src/test/java/org/apache/camel/itest/jmh/SimpleMockPlaceholderTest.java
@@ -0,0 +1,117 @@
+/*
+ * 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.itest.jmh;
+
+import java.util.Properties;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.ProducerTemplate;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.impl.DefaultCamelContext;
+import org.junit.Test;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.TearDown;
+import org.openjdk.jmh.infra.Blackhole;
+import org.openjdk.jmh.runner.Runner;
+import org.openjdk.jmh.runner.options.Options;
+import org.openjdk.jmh.runner.options.OptionsBuilder;
+import org.openjdk.jmh.runner.options.TimeValue;
+
+/**
+ * Tests a simple Camel route
+ */
+public class SimpleMockPlaceholderTest {
+
+    @Test
+    public void launchBenchmark() throws Exception {
+        Options opt = new OptionsBuilder()
+            // Specify which benchmarks to run.
+            // You can be more specific if you'd like to run only one benchmark per test.
+            .include(this.getClass().getName() + ".*")
+            // Set the following options as needed
+            .mode(Mode.All)
+            .timeUnit(TimeUnit.MICROSECONDS)
+            .warmupTime(TimeValue.seconds(1))
+            .warmupIterations(2)
+            .measurementTime(TimeValue.seconds(1))
+            .measurementIterations(2)
+            .threads(2)
+            .forks(1)
+            .shouldFailOnError(true)
+            .shouldDoGC(true)
+            .build();
+
+        new Runner(opt).run();
+    }
+
+    // The JMH samples are the best documentation for how to use it
+    // http://hg.openjdk.java.net/code-tools/jmh/file/tip/jmh-samples/src/main/java/org/openjdk/jmh/samples/
+    @State(Scope.Thread)
+    public static class BenchmarkState {
+        CamelContext camel;
+        ProducerTemplate producer;
+
+        @Setup(Level.Trial)
+        public void initialize() {
+            camel = new DefaultCamelContext();
+
+            Properties prop = new Properties();
+            prop.put("foolish", "log:foo?groupSize=10");
+            prop.put("mymock", "mock:result?retainFirst=0");
+            camel.getPropertiesComponent().setInitialProperties(prop);
+
+            try {
+                camel.addRoutes(new RouteBuilder() {
+                    @Override
+                    public void configure() throws Exception {
+                        from("direct:start").to("{{foolish}}").to("log:bar").to("{{mymock}}");
+                    }
+                });
+                camel.start();
+                producer = camel.createProducerTemplate();
+            } catch (Exception e) {
+                // ignore
+            }
+        }
+
+        @TearDown(Level.Trial)
+        public void close() {
+            try {
+                producer.stop();
+                camel.stop();
+            } catch (Exception e) {
+                // ignore
+            }
+        }
+
+    }
+
+    @Benchmark
+    @Measurement(batchSize = 1000)
+    public void simpleMockPlaceholderTest(BenchmarkState state, Blackhole bh) {
+        ProducerTemplate template = state.producer;
+        template.sendBody("direct:start", "Hello World");
+    }
+
+}
diff --git a/tests/camel-jmh/src/test/resources/log4j2.properties b/tests/camel-jmh/src/test/resources/log4j2.properties
index 6116fc7..2ae0380 100644
--- a/tests/camel-jmh/src/test/resources/log4j2.properties
+++ b/tests/camel-jmh/src/test/resources/log4j2.properties
@@ -17,7 +17,7 @@
 
 appender.file.type = File
 appender.file.name = file
-appender.file.fileName = target/camel-itest-test.log
+appender.file.fileName = target/camel-jmh-test.log
 appender.file.layout.type = PatternLayout
 appender.file.layout.pattern = %d [%-15.15t] %-5p %-30.30c{1} - %m%n
 appender.stdout.type = Console