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/12/23 14:08:20 UTC

[camel] 01/03: CAMEL-15986: camel-core - DynamicAware need to support component alias. CAMEL-15984: camel-jms to support DynamicAware with toD

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 be97224338b58d49fdcd30dfa52365e5b9887e7b
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Wed Dec 23 14:38:09 2020 +0100

    CAMEL-15986: camel-core - DynamicAware need to support component alias. CAMEL-15984: camel-jms to support DynamicAware with toD
---
 .../services/org/apache/camel/send-dynamic/jms     |   2 +
 .../camel/component/jms/JmsSendDynamicAware.java   | 125 +++++++++++++++++++++
 .../component/jms/JmsSendDynamicPostProcessor.java |  28 +++++
 .../camel/component/jms/JmsToDSendDynamicTest.java |  67 +++++++++++
 .../jms/JmsToDSendDynamicTwoDisabledTest.java      |  66 +++++++++++
 .../component/jms/JmsToDSendDynamicTwoTest.java    |  67 +++++++++++
 .../src/main/java/org/apache/camel/Component.java  |   7 ++
 .../camel/processor/SendDynamicProcessor.java      |  12 ++
 .../org/apache/camel/support/DefaultComponent.java |  28 +++--
 .../java/org/apache/camel/util/StringHelper.java   |  21 ++++
 .../org/apache/camel/util/StringHelperTest.java    |  11 +-
 11 files changed, 423 insertions(+), 11 deletions(-)

diff --git a/components/camel-jms/src/generated/resources/META-INF/services/org/apache/camel/send-dynamic/jms b/components/camel-jms/src/generated/resources/META-INF/services/org/apache/camel/send-dynamic/jms
new file mode 100644
index 0000000..8dd2069
--- /dev/null
+++ b/components/camel-jms/src/generated/resources/META-INF/services/org/apache/camel/send-dynamic/jms
@@ -0,0 +1,2 @@
+# Generated by camel build tools - do NOT edit this file!
+class=org.apache.camel.component.jms.JmsSendDynamicAware
diff --git a/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsSendDynamicAware.java b/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsSendDynamicAware.java
new file mode 100644
index 0000000..11eab1c
--- /dev/null
+++ b/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsSendDynamicAware.java
@@ -0,0 +1,125 @@
+/*
+ * 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.jms;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.spi.SendDynamicAware;
+import org.apache.camel.spi.annotations.SendDynamic;
+import org.apache.camel.support.service.ServiceSupport;
+import org.apache.camel.util.StringHelper;
+
+/**
+ * JMS based {@link org.apache.camel.spi.SendDynamicAware} which allows to optimise JMS components with the toD (dynamic
+ * to) DSL in Camel. This implementation optimises by allowing to provide dynamic parameters via
+ * {@link JmsConstants#JMS_DESTINATION_NAME} header instead of the endpoint uri. That allows to use a static endpoint
+ * and its producer to service dynamic requests.
+ */
+@SendDynamic("jms")
+public class JmsSendDynamicAware extends ServiceSupport implements SendDynamicAware {
+
+    private CamelContext camelContext;
+    private String scheme;
+
+    @Override
+    public String getScheme() {
+        return scheme;
+    }
+
+    @Override
+    public void setScheme(String scheme) {
+        this.scheme = scheme;
+    }
+
+    @Override
+    public CamelContext getCamelContext() {
+        return camelContext;
+    }
+
+    @Override
+    public void setCamelContext(CamelContext camelContext) {
+        this.camelContext = camelContext;
+    }
+
+    @Override
+    public boolean isOnlyDynamicQueryParameters() {
+        return true;
+    }
+
+    @Override
+    public boolean isLenientProperties() {
+        return false;
+    }
+
+    @Override
+    public DynamicAwareEntry prepare(Exchange exchange, String uri, String originalUri) throws Exception {
+        return new DynamicAwareEntry(uri, originalUri, null, null);
+    }
+
+    @Override
+    public String resolveStaticUri(Exchange exchange, DynamicAwareEntry entry) throws Exception {
+        String destination = parseDestinationName(entry.getUri());
+        if (destination != null) {
+            String originalDestination = parseDestinationName(entry.getOriginalUri());
+            if (!destination.equals(originalDestination)) {
+                // okay the destination was dynamic, so use the original as endpoint name
+                String answer = entry.getUri();
+                answer = StringHelper.replaceFirst(answer, destination, originalDestination);
+                return answer;
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public Processor createPreProcessor(Exchange exchange, DynamicAwareEntry entry) throws Exception {
+        if (exchange.getMessage().getHeader(JmsConstants.JMS_DESTINATION_NAME) != null) {
+            return null;
+        }
+
+        final String destinationName = parseDestinationName(entry.getUri());
+        return new Processor() {
+            @Override
+            public void process(Exchange exchange) throws Exception {
+                exchange.getMessage().setHeader(JmsConstants.JMS_DESTINATION_NAME, destinationName);
+            }
+        };
+    }
+
+    @Override
+    public Processor createPostProcessor(Exchange exchange, DynamicAwareEntry entry) throws Exception {
+        // no post processor is needed
+        return null;
+    }
+
+    private String parseDestinationName(String uri) {
+        // strip query
+        int pos = uri.indexOf('?');
+        if (pos != -1) {
+            uri = uri.substring(0, pos);
+        }
+        // destination name is after last colon
+        pos = uri.lastIndexOf(':');
+        if (pos != -1) {
+            return uri.substring(pos + 1);
+        } else {
+            return null;
+        }
+    }
+
+}
diff --git a/components/camel-jms/src/main/resources/org/apache/camel/component/jms/JmsSendDynamicPostProcessor.java b/components/camel-jms/src/main/resources/org/apache/camel/component/jms/JmsSendDynamicPostProcessor.java
new file mode 100644
index 0000000..2b15576
--- /dev/null
+++ b/components/camel-jms/src/main/resources/org/apache/camel/component/jms/JmsSendDynamicPostProcessor.java
@@ -0,0 +1,28 @@
+/*
+ * 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.jms;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+
+public class JmsSendDynamicPostProcessor implements Processor {
+
+    @Override
+    public void process(Exchange exchange) throws Exception {
+        exchange.getMessage().removeHeader(JmsConstants.JMS_DESTINATION_NAME);
+    }
+}
diff --git a/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsToDSendDynamicTest.java b/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsToDSendDynamicTest.java
new file mode 100644
index 0000000..fb75eed
--- /dev/null
+++ b/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsToDSendDynamicTest.java
@@ -0,0 +1,67 @@
+/*
+ * 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.jms;
+
+import javax.jms.ConnectionFactory;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.junit5.CamelTestSupport;
+import org.junit.jupiter.api.Test;
+
+import static org.apache.camel.component.jms.JmsComponent.jmsComponentAutoAcknowledge;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class JmsToDSendDynamicTest extends CamelTestSupport {
+
+    @Test
+    public void testToD() throws Exception {
+        template.sendBodyAndHeader("direct:start", "Hello bar", "where", "bar");
+        template.sendBodyAndHeader("direct:start", "Hello beer", "where", "beer");
+
+        // there should only be one activemq endpoint
+        long count = context.getEndpoints().stream().filter(e -> e.getEndpointUri().startsWith("activemq:")).count();
+        assertEquals(1, count, "There should only be 1 activemq endpoint");
+
+        // and the messages should be in the queues
+        String out = consumer.receiveBody("activemq:queue:bar", 2000, String.class);
+        assertEquals("Hello bar", out);
+        out = consumer.receiveBody("activemq:queue:beer", 2000, String.class);
+        assertEquals("Hello beer", out);
+    }
+
+    @Override
+    protected CamelContext createCamelContext() throws Exception {
+        CamelContext camelContext = super.createCamelContext();
+
+        ConnectionFactory connectionFactory = CamelJmsTestHelper.createPersistentConnectionFactory();
+        camelContext.addComponent("activemq", jmsComponentAutoAcknowledge(connectionFactory));
+
+        return camelContext;
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // route message dynamic using toD
+                from("direct:start").toD("activemq:queue:${header.where}");
+            }
+        };
+    }
+}
diff --git a/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsToDSendDynamicTwoDisabledTest.java b/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsToDSendDynamicTwoDisabledTest.java
new file mode 100644
index 0000000..b125115
--- /dev/null
+++ b/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsToDSendDynamicTwoDisabledTest.java
@@ -0,0 +1,66 @@
+/*
+ * 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.jms;
+
+import javax.jms.ConnectionFactory;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.junit5.CamelTestSupport;
+import org.junit.jupiter.api.Test;
+
+import static org.apache.camel.component.jms.JmsComponent.jmsComponentAutoAcknowledge;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class JmsToDSendDynamicTwoDisabledTest extends CamelTestSupport {
+
+    @Test
+    public void testToD() throws Exception {
+        template.sendBodyAndHeader("direct:start", "Hello bar", "where", "bar");
+        template.sendBodyAndHeader("direct:start", "Hello beer", "where", "beer");
+        template.sendBodyAndHeader("direct:start", "Hello gin", "where", "gin");
+
+        template.sendBodyAndHeader("direct:start2", "Hello beer", "where2", "beer");
+        template.sendBodyAndHeader("direct:start2", "Hello whiskey", "where2", "whiskey");
+
+        // there should be 4 activemq endpoint
+        long count = context.getEndpoints().stream().filter(e -> e.getEndpointUri().startsWith("activemq:")).count();
+        assertEquals(4, count, "There should be 4 activemq endpoint");
+    }
+
+    @Override
+    protected CamelContext createCamelContext() throws Exception {
+        CamelContext camelContext = super.createCamelContext();
+
+        ConnectionFactory connectionFactory = CamelJmsTestHelper.createPersistentConnectionFactory();
+        camelContext.addComponent("activemq", jmsComponentAutoAcknowledge(connectionFactory));
+
+        return camelContext;
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // route message dynamic using toD but turn off send dynamic aware
+                from("direct:start").toD().allowOptimisedComponents(false).uri("activemq:queue:${header.where}");
+                from("direct:start2").toD().allowOptimisedComponents(false).uri("activemq:queue:${header.where2}");
+            }
+        };
+    }
+}
diff --git a/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsToDSendDynamicTwoTest.java b/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsToDSendDynamicTwoTest.java
new file mode 100644
index 0000000..83f92ce
--- /dev/null
+++ b/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsToDSendDynamicTwoTest.java
@@ -0,0 +1,67 @@
+/*
+ * 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.jms;
+
+import javax.jms.ConnectionFactory;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.test.junit5.CamelTestSupport;
+import org.junit.jupiter.api.Test;
+
+import static org.apache.camel.component.jms.JmsComponent.jmsComponentAutoAcknowledge;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class JmsToDSendDynamicTwoTest extends CamelTestSupport {
+
+    @Test
+    public void testToD() throws Exception {
+        template.sendBodyAndHeader("direct:start", "Hello bar", "where", "bar");
+        template.sendBodyAndHeader("direct:start", "Hello beer", "where", "beer");
+        template.sendBodyAndHeader("direct:start", "Hello gin", "where", "gin");
+
+        template.sendBodyAndHeader("direct:start2", "Hello beer", "where2", "beer");
+        template.sendBodyAndHeader("direct:start2", "Hello whiskey", "where2", "whiskey");
+
+        // there should be 2 activemq endpoint
+        long count = context.getEndpoints().stream().filter(e -> e.getEndpointUri().startsWith("activemq:")).count();
+        assertEquals(2, count, "There should only be 2 activemq endpoint");
+    }
+
+    @Override
+    protected CamelContext createCamelContext() throws Exception {
+        CamelContext camelContext = super.createCamelContext();
+
+        ConnectionFactory connectionFactory = CamelJmsTestHelper.createPersistentConnectionFactory();
+        camelContext.addComponent("activemq", jmsComponentAutoAcknowledge(connectionFactory));
+
+        return camelContext;
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                // route message dynamic using toD
+                from("direct:start").toD("activemq:queue:${header.where}");
+
+                from("direct:start2").toD("activemq:queue:${header.where2}");
+            }
+        };
+    }
+}
diff --git a/core/camel-api/src/main/java/org/apache/camel/Component.java b/core/camel-api/src/main/java/org/apache/camel/Component.java
index 298aa4e..3d0d548 100644
--- a/core/camel-api/src/main/java/org/apache/camel/Component.java
+++ b/core/camel-api/src/main/java/org/apache/camel/Component.java
@@ -115,4 +115,11 @@ public interface Component extends CamelContextAware, Service {
         return object;
     }
 
+    /**
+     * Gets the default name of the component.
+     */
+    default String getDefaultName() {
+        return null;
+    }
+
 }
diff --git a/core/camel-core-processor/src/main/java/org/apache/camel/processor/SendDynamicProcessor.java b/core/camel-core-processor/src/main/java/org/apache/camel/processor/SendDynamicProcessor.java
index da416aa..9f26dee 100644
--- a/core/camel-core-processor/src/main/java/org/apache/camel/processor/SendDynamicProcessor.java
+++ b/core/camel-core-processor/src/main/java/org/apache/camel/processor/SendDynamicProcessor.java
@@ -19,6 +19,7 @@ package org.apache.camel.processor;
 import org.apache.camel.AsyncCallback;
 import org.apache.camel.CamelContext;
 import org.apache.camel.CamelContextAware;
+import org.apache.camel.Component;
 import org.apache.camel.Endpoint;
 import org.apache.camel.Exchange;
 import org.apache.camel.ExchangePattern;
@@ -320,6 +321,17 @@ public class SendDynamicProcessor extends AsyncProcessorSupport implements IdAwa
                     // find out if the component can be optimised for send-dynamic
                     SendDynamicAwareResolver resolver = new SendDynamicAwareResolver();
                     dynamicAware = resolver.resolve(camelContext, scheme);
+                    if (dynamicAware == null) {
+                        // okay fallback and try with default component name
+                        Component comp = camelContext.getComponent(scheme, false, isAutoStartupComponents());
+                        if (comp != null) {
+                            String defaultScheme = comp.getDefaultName();
+                            if (!scheme.equals(defaultScheme)) {
+                                dynamicAware = resolver.resolve(camelContext, defaultScheme);
+                                dynamicAware.setScheme(scheme);
+                            }
+                        }
+                    }
                     if (dynamicAware != null) {
                         if (LOG.isDebugEnabled()) {
                             LOG.debug("Detected SendDynamicAware component: {} optimising toD: {}", scheme,
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 cf3bdf1..e7e1982 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
@@ -62,6 +62,7 @@ public abstract class DefaultComponent extends ServiceSupport implements Compone
 
     private volatile PropertyConfigurer componentPropertyConfigurer;
     private volatile PropertyConfigurer endpointPropertyConfigurer;
+    private volatile String defaultName;
     private final List<Supplier<ComponentExtension>> extensions = new ArrayList<>();
     private CamelContext camelContext;
 
@@ -354,19 +355,28 @@ public abstract class DefaultComponent extends ServiceSupport implements Compone
     }
 
     @Override
+    public String getDefaultName() {
+        return defaultName;
+    }
+
+    @Override
     protected void doBuild() throws Exception {
-        org.apache.camel.spi.annotations.Component ann
-                = ObjectHelper.getAnnotation(this, org.apache.camel.spi.annotations.Component.class);
-        if (ann != null) {
-            String name = ann.value();
-            // just grab first scheme name if the component has scheme alias (eg http,https)
-            if (name.contains(",")) {
-                name = StringHelper.before(name, ",");
+        if (defaultName == null) {
+            org.apache.camel.spi.annotations.Component ann
+                    = ObjectHelper.getAnnotation(this, org.apache.camel.spi.annotations.Component.class);
+            if (ann != null) {
+                defaultName = ann.value();
+                // just grab first scheme name if the component has scheme alias (eg http,https)
+                if (defaultName.contains(",")) {
+                    defaultName = StringHelper.before(defaultName, ",");
+                }
             }
-            final String componentConfigurerName = name + "-component-configurer";
+        }
+        if (defaultName != null) {
+            final String componentConfigurerName = defaultName + "-component-configurer";
             componentPropertyConfigurer = getCamelContext().adapt(ExtendedCamelContext.class).getConfigurerResolver()
                     .resolvePropertyConfigurer(componentConfigurerName, getCamelContext());
-            final String endpointConfigurerName = name + "-endpoint-configurer";
+            final String endpointConfigurerName = defaultName + "-endpoint-configurer";
             endpointPropertyConfigurer = getCamelContext().adapt(ExtendedCamelContext.class).getConfigurerResolver()
                     .resolvePropertyConfigurer(endpointConfigurerName, getCamelContext());
         }
diff --git a/core/camel-util/src/main/java/org/apache/camel/util/StringHelper.java b/core/camel-util/src/main/java/org/apache/camel/util/StringHelper.java
index 8199401..34607c5 100644
--- a/core/camel-util/src/main/java/org/apache/camel/util/StringHelper.java
+++ b/core/camel-util/src/main/java/org/apache/camel/util/StringHelper.java
@@ -310,6 +310,27 @@ public final class StringHelper {
     }
 
     /**
+     * Replaces the first from token in the given input string.
+     * <p/>
+     * This implementation is not recursive, not does it check for tokens in the replacement string.
+     *
+     * @param  input                    the input string
+     * @param  from                     the from string, must <b>not</b> be <tt>null</tt> or empty
+     * @param  to                       the replacement string, must <b>not</b> be empty
+     * @return                          the replaced string, or the input string if no replacement was needed
+     * @throws IllegalArgumentException if the input arguments is invalid
+     */
+    public static String replaceFirst(String input, String from, String to) {
+        int pos = input.indexOf(from);
+        if (pos != -1) {
+            int len = from.length();
+            return input.substring(0, pos) + to + input.substring(pos + len);
+        } else {
+            return input;
+        }
+    }
+
+    /**
      * Creates a json tuple with the given name/value pair.
      *
      * @param  name  the name
diff --git a/core/camel-util/src/test/java/org/apache/camel/util/StringHelperTest.java b/core/camel-util/src/test/java/org/apache/camel/util/StringHelperTest.java
index f2487c5..a75e476 100644
--- a/core/camel-util/src/test/java/org/apache/camel/util/StringHelperTest.java
+++ b/core/camel-util/src/test/java/org/apache/camel/util/StringHelperTest.java
@@ -18,8 +18,7 @@ package org.apache.camel.util;
 
 import org.junit.jupiter.api.Test;
 
-import static org.apache.camel.util.StringHelper.camelCaseToDash;
-import static org.apache.camel.util.StringHelper.splitWords;
+import static org.apache.camel.util.StringHelper.*;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class StringHelperTest {
@@ -67,4 +66,12 @@ public class StringHelperTest {
         assertEquals(1, arr.length);
         assertEquals("hello", arr[0]);
     }
+
+    @Test
+    public void testReplaceFirst() throws Exception {
+        assertEquals("jms:queue:bar", replaceFirst("jms:queue:bar", "foo", "bar"));
+        assertEquals("jms:queue:bar", replaceFirst("jms:queue:foo", "foo", "bar"));
+        assertEquals("jms:queue:bar?blah=123", replaceFirst("jms:queue:foo?blah=123", "foo", "bar"));
+        assertEquals("jms:queue:bar?blah=foo", replaceFirst("jms:queue:foo?blah=foo", "foo", "bar"));
+    }
 }