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 2023/12/30 15:15:07 UTC

(camel) 10/25: CAMEL-19749: Add variables as concept to Camel

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

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

commit c5f19f53fe35bb12024ceb8bb8522a2ef0151e7a
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Fri Dec 29 18:30:12 2023 +0100

    CAMEL-19749: Add variables as concept to Camel
---
 .../main/java/org/apache/camel/CamelContext.java   |  19 ++++
 .../src/main/java/org/apache/camel/Exchange.java   |   6 +-
 .../java/org/apache/camel/ExchangeExtension.java   |   1 +
 .../org/apache/camel/spi/VariableRepository.java   |  55 ++++++++++
 .../camel/spi/VariableRepositoryFactory.java       |  32 ++++++
 .../camel/impl/engine/AbstractCamelContext.java    |  26 +++++
 .../engine/DefaultVariableRepositoryFactory.java   |  75 ++++++++++++++
 .../camel/impl/engine/SimpleCamelContext.java      |   6 ++
 .../camel/processor/SetVariableProcessor.java      |  40 +++++++-
 .../camel/processor/CustomGlobalVariableTest.java  | 112 +++++++++++++++++++++
 .../camel/processor/SetGlobalVariableTest.java     |  74 ++++++++++++++
 .../mbean/ManagedVariableRepositoryMBean.java      |  26 +++++
 .../management/JmxManagementLifecycleStrategy.java |   4 +
 .../mbean/ManagedVariableRepository.java           |  41 ++++++++
 .../management/ManagedNonManagedServiceTest.java   |   2 +-
 ...edProducerRouteAddRemoveRegisterAlwaysTest.java |   2 +-
 .../management/ManagedRouteAddRemoveTest.java      |   2 +-
 .../org/apache/camel/support/AbstractExchange.java |  86 ++++++++--------
 .../camel/support/DefaultPooledExchange.java       |   3 -
 .../camel/support/ExchangeVariableRepository.java  |  87 ++++++++++++++++
 .../camel/support/ExtendedExchangeExtension.java   |   3 +-
 .../camel/support/GlobalVariableRepository.java    |  58 +++++++++++
 .../camel/support/builder/ExpressionBuilder.java   |   2 +-
 .../processor/DefaultExchangeFormatter.java        |   8 +-
 .../java/org/apache/camel/util/StringHelper.java   |  18 ++++
 25 files changed, 728 insertions(+), 60 deletions(-)

diff --git a/core/camel-api/src/main/java/org/apache/camel/CamelContext.java b/core/camel-api/src/main/java/org/apache/camel/CamelContext.java
index 22cfc3ed793..8fadd42919f 100644
--- a/core/camel-api/src/main/java/org/apache/camel/CamelContext.java
+++ b/core/camel-api/src/main/java/org/apache/camel/CamelContext.java
@@ -806,6 +806,25 @@ public interface CamelContext extends CamelContextLifecycle, RuntimeConfiguratio
      */
     String resolvePropertyPlaceholders(String text);
 
+    /**
+     * To get a variable by name.
+     *
+     * @param  name the variable name. Can be prefixed with repo-id:name to lookup the variable from a specific
+     *              repository. If no repo-id is provided, then global repository will be used.
+     * @return      the variable, or <tt>null</tt> if not found.
+     */
+    Object getVariable(String name);
+
+    /**
+     * To get a variable by name and covert to the given type.
+     *
+     * @param  name the variable name. Can be prefixed with repo-id:name to lookup the variable from a specific
+     *              repository. If no repo-id is provided, then global repository will be used.
+     * @param  type the type to convert the variable to
+     * @return      the variable, or <tt>null</tt> if not found.
+     */
+    <T> T getVariable(String name, Class<T> type);
+
     /**
      * Returns the configured properties component or create one if none has been configured.
      *
diff --git a/core/camel-api/src/main/java/org/apache/camel/Exchange.java b/core/camel-api/src/main/java/org/apache/camel/Exchange.java
index a91f9865b6c..1946492717d 100644
--- a/core/camel-api/src/main/java/org/apache/camel/Exchange.java
+++ b/core/camel-api/src/main/java/org/apache/camel/Exchange.java
@@ -449,7 +449,7 @@ public interface Exchange {
     <T> T getVariable(String name, Object defaultValue, Class<T> type);
 
     /**
-     * Sets a varialbe on the exchange
+     * Sets a variable on the exchange
      *
      * @param name  of the variable
      * @param value the value of the variable
@@ -460,7 +460,7 @@ public interface Exchange {
      * Removes the given variable
      *
      * @param  name of the variable
-     * @return      the old value of the variable
+     * @return      the old value of the variable, or <tt>null</tt> if there was no variable for the given name
      */
     Object removeVariable(String name);
 
@@ -485,7 +485,7 @@ public interface Exchange {
     /**
      * Returns the variables
      *
-     * @return the variables in a Map
+     * @return the variables in a Map.
      */
     Map<String, Object> getVariables();
 
diff --git a/core/camel-api/src/main/java/org/apache/camel/ExchangeExtension.java b/core/camel-api/src/main/java/org/apache/camel/ExchangeExtension.java
index 7ea4a144193..f076041c70c 100644
--- a/core/camel-api/src/main/java/org/apache/camel/ExchangeExtension.java
+++ b/core/camel-api/src/main/java/org/apache/camel/ExchangeExtension.java
@@ -272,4 +272,5 @@ public interface ExchangeExtension {
      * @return         A new Exchange instance
      */
     Exchange createCopyWithProperties(CamelContext context);
+
 }
diff --git a/core/camel-api/src/main/java/org/apache/camel/spi/VariableRepository.java b/core/camel-api/src/main/java/org/apache/camel/spi/VariableRepository.java
new file mode 100644
index 00000000000..91d78bfa52f
--- /dev/null
+++ b/core/camel-api/src/main/java/org/apache/camel/spi/VariableRepository.java
@@ -0,0 +1,55 @@
+/*
+ * 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;
+
+import org.apache.camel.StaticService;
+
+/**
+ * Repository for storing and accessing variables.
+ */
+public interface VariableRepository extends StaticService {
+
+    /**
+     * The id of this repository.
+     */
+    String getId();
+
+    /**
+     * Returns a variable by name
+     *
+     * @param  name the name of the variable
+     * @return      the value of the given variable or <tt>null</tt> if there is no variable for the given name
+     */
+    Object getVariable(String name);
+
+    /**
+     * Sets a variable
+     *
+     * @param name  of the variable
+     * @param value the value of the variable
+     */
+    void setVariable(String name, Object value);
+
+    /**
+     * Removes the given variable
+     *
+     * @param  name of the variable
+     * @return      the old value of the variable, or <tt>null</tt> if there was no variable for the given name
+     */
+    Object removeVariable(String name);
+
+}
diff --git a/core/camel-api/src/main/java/org/apache/camel/spi/VariableRepositoryFactory.java b/core/camel-api/src/main/java/org/apache/camel/spi/VariableRepositoryFactory.java
new file mode 100644
index 00000000000..ba7aefeb473
--- /dev/null
+++ b/core/camel-api/src/main/java/org/apache/camel/spi/VariableRepositoryFactory.java
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * Factory for {@link VariableRepository}.
+ */
+public interface VariableRepositoryFactory {
+
+    /**
+     * Gets the {@link VariableRepository} for the given id
+     *
+     * @param  id the repository id
+     * @return    the repository or <tt>null</tt> if none found
+     */
+    VariableRepository getVariableRepository(String id);
+
+}
diff --git a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/AbstractCamelContext.java b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/AbstractCamelContext.java
index 5f2060182d7..461444c6590 100644
--- a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/AbstractCamelContext.java
+++ b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/AbstractCamelContext.java
@@ -166,6 +166,8 @@ import org.apache.camel.spi.UriFactoryResolver;
 import org.apache.camel.spi.UuidGenerator;
 import org.apache.camel.spi.Validator;
 import org.apache.camel.spi.ValidatorRegistry;
+import org.apache.camel.spi.VariableRepository;
+import org.apache.camel.spi.VariableRepositoryFactory;
 import org.apache.camel.support.CamelContextHelper;
 import org.apache.camel.support.EndpointHelper;
 import org.apache.camel.support.EventHelper;
@@ -340,6 +342,7 @@ public abstract class AbstractCamelContext extends BaseService
         camelContextExtension.addContextPlugin(FactoryFinderResolver.class, createFactoryFinderResolver());
         camelContextExtension.addContextPlugin(PackageScanClassResolver.class, createPackageScanClassResolver());
         camelContextExtension.addContextPlugin(PackageScanResourceResolver.class, createPackageScanResourceResolver());
+        camelContextExtension.addContextPlugin(VariableRepositoryFactory.class, createVariableRepositoryFactory());
         camelContextExtension.lazyAddContextPlugin(ModelineFactory.class, this::createModelineFactory);
         camelContextExtension.lazyAddContextPlugin(ModelJAXBContextFactory.class, this::createModelJAXBContextFactory);
         camelContextExtension.addContextPlugin(DataFormatResolver.class, createDataFormatResolver());
@@ -1557,6 +1560,27 @@ public abstract class AbstractCamelContext extends BaseService
         return camelContextExtension.resolvePropertyPlaceholders(text, false);
     }
 
+    @Override
+    public Object getVariable(String name) {
+        String id = StringHelper.before(name, ":", "global");
+        name = StringHelper.after(name, ":", name);
+        VariableRepository repo
+                = camelContextExtension.getContextPlugin(VariableRepositoryFactory.class).getVariableRepository(id);
+        if (repo != null) {
+            return repo.getVariable(name);
+        }
+        return null;
+    }
+
+    @Override
+    public <T> T getVariable(String name, Class<T> type) {
+        Object value = getVariable(name);
+        if (value != null) {
+            return getTypeConverter().convertTo(type, value);
+        }
+        return null;
+    }
+
     @Override
     public TypeConverter getTypeConverter() {
         return camelContextExtension.getTypeConverter();
@@ -3996,6 +4020,8 @@ public abstract class AbstractCamelContext extends BaseService
 
     protected abstract ValidatorRegistry<ValidatorKey> createValidatorRegistry();
 
+    protected abstract VariableRepositoryFactory createVariableRepositoryFactory();
+
     protected RestConfiguration createRestConfiguration() {
         // lookup a global which may have been on a container such spring-boot / CDI / etc.
         RestConfiguration conf
diff --git a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultVariableRepositoryFactory.java b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultVariableRepositoryFactory.java
new file mode 100644
index 00000000000..524192b909f
--- /dev/null
+++ b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultVariableRepositoryFactory.java
@@ -0,0 +1,75 @@
+/*
+ * 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.impl.engine;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.StaticService;
+import org.apache.camel.spi.VariableRepository;
+import org.apache.camel.spi.VariableRepositoryFactory;
+import org.apache.camel.support.CamelContextHelper;
+import org.apache.camel.support.GlobalVariableRepository;
+import org.apache.camel.support.service.ServiceSupport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Default {@link VariableRepositoryFactory}.
+ */
+public class DefaultVariableRepositoryFactory extends ServiceSupport implements VariableRepositoryFactory, StaticService {
+
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultVariableRepositoryFactory.class);
+
+    private final CamelContext camelContext;
+    private VariableRepository global;
+
+    public DefaultVariableRepositoryFactory(CamelContext camelContext) {
+        this.camelContext = camelContext;
+    }
+
+    @Override
+    public VariableRepository getVariableRepository(String id) {
+        if ("global".equals(id)) {
+            return global;
+        }
+
+        // otherwise lookup in registry if the repo exists
+        VariableRepository repo = CamelContextHelper.lookup(camelContext, id, VariableRepository.class);
+        if (repo != null) {
+            return repo;
+        }
+
+        return null;
+    }
+
+    @Override
+    protected void doStart() throws Exception {
+        super.doStart();
+
+        VariableRepository repo = CamelContextHelper.findSingleByType(camelContext, VariableRepository.class);
+        if (repo != null) {
+            LOG.info("Using VariableRepository: {} as global repository", repo.getId());
+            global = repo;
+        } else {
+            global = new GlobalVariableRepository();
+        }
+
+        if (!camelContext.hasService(global)) {
+            camelContext.addService(global);
+        }
+    }
+
+}
diff --git a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/SimpleCamelContext.java b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/SimpleCamelContext.java
index 04bc7c475f0..4b013e81e81 100644
--- a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/SimpleCamelContext.java
+++ b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/SimpleCamelContext.java
@@ -91,6 +91,7 @@ import org.apache.camel.spi.UnitOfWorkFactory;
 import org.apache.camel.spi.UriFactoryResolver;
 import org.apache.camel.spi.UuidGenerator;
 import org.apache.camel.spi.ValidatorRegistry;
+import org.apache.camel.spi.VariableRepositoryFactory;
 import org.apache.camel.support.DefaultRegistry;
 import org.apache.camel.support.DefaultUuidGenerator;
 import org.apache.camel.support.NormalizedUri;
@@ -720,6 +721,11 @@ public class SimpleCamelContext extends AbstractCamelContext {
         return new DefaultValidatorRegistry(getCamelContextReference());
     }
 
+    @Override
+    protected VariableRepositoryFactory createVariableRepositoryFactory() {
+        return new DefaultVariableRepositoryFactory(getCamelContextReference());
+    }
+
     @Override
     protected TransformerRegistry<TransformerKey> createTransformerRegistry() {
         return new DefaultTransformerRegistry(getCamelContextReference());
diff --git a/core/camel-core-processor/src/main/java/org/apache/camel/processor/SetVariableProcessor.java b/core/camel-core-processor/src/main/java/org/apache/camel/processor/SetVariableProcessor.java
index 0f8bb726253..f0690902e27 100644
--- a/core/camel-core-processor/src/main/java/org/apache/camel/processor/SetVariableProcessor.java
+++ b/core/camel-core-processor/src/main/java/org/apache/camel/processor/SetVariableProcessor.java
@@ -17,22 +17,30 @@
 package org.apache.camel.processor;
 
 import org.apache.camel.AsyncCallback;
+import org.apache.camel.CamelContext;
+import org.apache.camel.CamelContextAware;
 import org.apache.camel.Exchange;
 import org.apache.camel.Expression;
 import org.apache.camel.Traceable;
 import org.apache.camel.spi.IdAware;
 import org.apache.camel.spi.RouteIdAware;
+import org.apache.camel.spi.VariableRepository;
+import org.apache.camel.spi.VariableRepositoryFactory;
 import org.apache.camel.support.AsyncProcessorSupport;
 import org.apache.camel.util.ObjectHelper;
+import org.apache.camel.util.StringHelper;
 
 /**
  * A processor which sets the variable with an {@link Expression}
  */
-public class SetVariableProcessor extends AsyncProcessorSupport implements Traceable, IdAware, RouteIdAware {
+public class SetVariableProcessor extends AsyncProcessorSupport implements Traceable, IdAware, RouteIdAware, CamelContextAware {
+
+    private CamelContext camelContext;
     private String id;
     private String routeId;
     private final Expression variableName;
     private final Expression expression;
+    private VariableRepositoryFactory factory;
 
     public SetVariableProcessor(Expression variableName, Expression expression) {
         this.variableName = variableName;
@@ -41,6 +49,16 @@ public class SetVariableProcessor extends AsyncProcessorSupport implements Trace
         ObjectHelper.notNull(expression, "expression");
     }
 
+    @Override
+    public CamelContext getCamelContext() {
+        return camelContext;
+    }
+
+    @Override
+    public void setCamelContext(CamelContext camelContext) {
+        this.camelContext = camelContext;
+    }
+
     @Override
     public boolean process(Exchange exchange, AsyncCallback callback) {
         try {
@@ -53,7 +71,19 @@ public class SetVariableProcessor extends AsyncProcessorSupport implements Trace
             }
 
             String key = variableName.evaluate(exchange, String.class);
-            exchange.setVariable(key, newVariable);
+            String id = StringHelper.before(key, ":");
+            if (id != null) {
+                VariableRepository repo = factory.getVariableRepository(id);
+                if (repo != null) {
+                    key = StringHelper.after(key, ":");
+                    repo.setVariable(key, newVariable);
+                } else {
+                    exchange.setException(
+                            new IllegalArgumentException("VariableRepository with id: " + id + " does not exists"));
+                }
+            } else {
+                exchange.setVariable(key, newVariable);
+            }
         } catch (Exception e) {
             exchange.setException(e);
         }
@@ -62,6 +92,12 @@ public class SetVariableProcessor extends AsyncProcessorSupport implements Trace
         return true;
     }
 
+    @Override
+    protected void doBuild() throws Exception {
+        super.doBuild();
+        factory = getCamelContext().getCamelContextExtension().getContextPlugin(VariableRepositoryFactory.class);
+    }
+
     @Override
     public String toString() {
         return id;
diff --git a/core/camel-core/src/test/java/org/apache/camel/processor/CustomGlobalVariableTest.java b/core/camel-core/src/test/java/org/apache/camel/processor/CustomGlobalVariableTest.java
new file mode 100644
index 00000000000..82192e1efbf
--- /dev/null
+++ b/core/camel-core/src/test/java/org/apache/camel/processor/CustomGlobalVariableTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.processor;
+
+import java.util.List;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Exchange;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.spi.VariableRepository;
+import org.apache.camel.support.service.ServiceSupport;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+public class CustomGlobalVariableTest extends ContextTestSupport {
+    private MockEndpoint end;
+    private String variableName = "foo";
+    private String expectedVariableValue = "bar";
+
+    @Override
+    protected CamelContext createCamelContext() throws Exception {
+        CamelContext context = super.createCamelContext();
+        context.getRegistry().bind("myGlobal", new MyGlobalRepo());
+        return context;
+    }
+
+    @Test
+    public void testSetExchangeVariableMidRoute() throws Exception {
+        assertNull(context.getVariable(variableName));
+
+        end.expectedMessageCount(1);
+
+        template.sendBody("direct:start", "<blah/>");
+
+        // make sure we got the message
+        assertMockEndpointsSatisfied();
+
+        // lets get the variable value
+        List<Exchange> exchanges = end.getExchanges();
+        Exchange exchange = exchanges.get(0);
+        String actualVariableValue = exchange.getVariable(variableName, String.class);
+        // should be stored on global so null
+        assertNull(actualVariableValue);
+
+        // should be stored as global variable
+        assertEquals("!" + expectedVariableValue + "!", context.getVariable(variableName));
+    }
+
+    @Override
+    @BeforeEach
+    public void setUp() throws Exception {
+        super.setUp();
+        end = getMockEndpoint("mock:end");
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+                // stored as global variable
+                from("direct:start").setVariable("global:" + variableName).constant(expectedVariableValue).to("mock:end");
+            }
+        };
+    }
+
+    private static class MyGlobalRepo extends ServiceSupport implements VariableRepository {
+
+        private Object value;
+
+        @Override
+        public String getId() {
+            return "myGlobal";
+        }
+
+        @Override
+        public Object getVariable(String name) {
+            if (value != null) {
+                return "!" + value + "!";
+            }
+            return null;
+        }
+
+        @Override
+        public void setVariable(String name, Object value) {
+            this.value = value;
+        }
+
+        @Override
+        public Object removeVariable(String name) {
+            return null;
+        }
+    }
+}
diff --git a/core/camel-core/src/test/java/org/apache/camel/processor/SetGlobalVariableTest.java b/core/camel-core/src/test/java/org/apache/camel/processor/SetGlobalVariableTest.java
new file mode 100644
index 00000000000..7870bd292f1
--- /dev/null
+++ b/core/camel-core/src/test/java/org/apache/camel/processor/SetGlobalVariableTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.processor;
+
+import java.util.List;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Exchange;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
+public class SetGlobalVariableTest extends ContextTestSupport {
+    private MockEndpoint end;
+    private String variableName = "foo";
+    private String expectedVariableValue = "bar";
+
+    @Test
+    public void testSetExchangeVariableMidRoute() throws Exception {
+        assertNull(context.getVariable(variableName));
+
+        end.expectedMessageCount(1);
+
+        template.sendBody("direct:start", "<blah/>");
+
+        // make sure we got the message
+        assertMockEndpointsSatisfied();
+
+        // lets get the variable value
+        List<Exchange> exchanges = end.getExchanges();
+        Exchange exchange = exchanges.get(0);
+        String actualVariableValue = exchange.getVariable(variableName, String.class);
+        // should be stored on global so null
+        assertNull(actualVariableValue);
+
+        // should be stored as global variable
+        assertEquals(expectedVariableValue, context.getVariable(variableName));
+    }
+
+    @Override
+    @BeforeEach
+    public void setUp() throws Exception {
+        super.setUp();
+        end = getMockEndpoint("mock:end");
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+                // stored as global variable
+                from("direct:start").setVariable("global:" + variableName).constant(expectedVariableValue).to("mock:end");
+            }
+        };
+    }
+}
diff --git a/core/camel-management-api/src/main/java/org/apache/camel/api/management/mbean/ManagedVariableRepositoryMBean.java b/core/camel-management-api/src/main/java/org/apache/camel/api/management/mbean/ManagedVariableRepositoryMBean.java
new file mode 100644
index 00000000000..d9e4ea37583
--- /dev/null
+++ b/core/camel-management-api/src/main/java/org/apache/camel/api/management/mbean/ManagedVariableRepositoryMBean.java
@@ -0,0 +1,26 @@
+/*
+ * 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.api.management.mbean;
+
+import org.apache.camel.api.management.ManagedAttribute;
+
+public interface ManagedVariableRepositoryMBean extends ManagedServiceMBean {
+
+    @ManagedAttribute(description = "Repository ID")
+    String getId();
+
+}
diff --git a/core/camel-management/src/main/java/org/apache/camel/management/JmxManagementLifecycleStrategy.java b/core/camel-management/src/main/java/org/apache/camel/management/JmxManagementLifecycleStrategy.java
index ec17575664e..83192a799f7 100644
--- a/core/camel-management/src/main/java/org/apache/camel/management/JmxManagementLifecycleStrategy.java
+++ b/core/camel-management/src/main/java/org/apache/camel/management/JmxManagementLifecycleStrategy.java
@@ -74,6 +74,7 @@ import org.apache.camel.management.mbean.ManagedTracer;
 import org.apache.camel.management.mbean.ManagedTransformerRegistry;
 import org.apache.camel.management.mbean.ManagedTypeConverterRegistry;
 import org.apache.camel.management.mbean.ManagedValidatorRegistry;
+import org.apache.camel.management.mbean.ManagedVariableRepository;
 import org.apache.camel.model.InterceptDefinition;
 import org.apache.camel.model.OnCompletionDefinition;
 import org.apache.camel.model.OnExceptionDefinition;
@@ -106,6 +107,7 @@ import org.apache.camel.spi.TransformerRegistry;
 import org.apache.camel.spi.TypeConverterRegistry;
 import org.apache.camel.spi.UnitOfWork;
 import org.apache.camel.spi.ValidatorRegistry;
+import org.apache.camel.spi.VariableRepository;
 import org.apache.camel.support.TimerListenerManager;
 import org.apache.camel.support.service.ServiceSupport;
 import org.apache.camel.throttling.ThrottlingExceptionRoutePolicy;
@@ -565,6 +567,8 @@ public class JmxManagementLifecycleStrategy extends ServiceSupport implements Li
             answer = new ManagedTransformerRegistry(context, transformerRegistry);
         } else if (service instanceof ValidatorRegistry<?> validatorRegistry) {
             answer = new ManagedValidatorRegistry(context, validatorRegistry);
+        } else if (service instanceof VariableRepository variableRepository) {
+            answer = new ManagedVariableRepository(context, variableRepository);
         } else if (service instanceof CamelClusterService) {
             answer = getManagementObjectStrategy().getManagedObjectForClusterService(context, (CamelClusterService) service);
         } else if (service != null) {
diff --git a/core/camel-management/src/main/java/org/apache/camel/management/mbean/ManagedVariableRepository.java b/core/camel-management/src/main/java/org/apache/camel/management/mbean/ManagedVariableRepository.java
new file mode 100644
index 00000000000..75c9569ff3a
--- /dev/null
+++ b/core/camel-management/src/main/java/org/apache/camel/management/mbean/ManagedVariableRepository.java
@@ -0,0 +1,41 @@
+/*
+ * 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.management.mbean;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.api.management.ManagedResource;
+import org.apache.camel.api.management.mbean.ManagedVariableRepositoryMBean;
+import org.apache.camel.spi.VariableRepository;
+
+@ManagedResource(description = "Managed VariableRepository")
+public class ManagedVariableRepository extends ManagedService implements ManagedVariableRepositoryMBean {
+    private final VariableRepository variableRepository;
+
+    public ManagedVariableRepository(CamelContext context, VariableRepository variableRepository) {
+        super(context, variableRepository);
+        this.variableRepository = variableRepository;
+    }
+
+    public VariableRepository getVariableRepository() {
+        return variableRepository;
+    }
+
+    @Override
+    public String getId() {
+        return variableRepository.getId();
+    }
+}
diff --git a/core/camel-management/src/test/java/org/apache/camel/management/ManagedNonManagedServiceTest.java b/core/camel-management/src/test/java/org/apache/camel/management/ManagedNonManagedServiceTest.java
index 4450b8511fa..29e6e24dcf8 100644
--- a/core/camel-management/src/test/java/org/apache/camel/management/ManagedNonManagedServiceTest.java
+++ b/core/camel-management/src/test/java/org/apache/camel/management/ManagedNonManagedServiceTest.java
@@ -34,7 +34,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
 @DisabledOnOs(OS.AIX)
 public class ManagedNonManagedServiceTest extends ManagementTestSupport {
 
-    private static final int SERVICES = 14;
+    private static final int SERVICES = 15;
 
     @Test
     public void testService() throws Exception {
diff --git a/core/camel-management/src/test/java/org/apache/camel/management/ManagedProducerRouteAddRemoveRegisterAlwaysTest.java b/core/camel-management/src/test/java/org/apache/camel/management/ManagedProducerRouteAddRemoveRegisterAlwaysTest.java
index 57a0b4c33d5..56b9275cc96 100644
--- a/core/camel-management/src/test/java/org/apache/camel/management/ManagedProducerRouteAddRemoveRegisterAlwaysTest.java
+++ b/core/camel-management/src/test/java/org/apache/camel/management/ManagedProducerRouteAddRemoveRegisterAlwaysTest.java
@@ -36,7 +36,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 @DisabledOnOs(OS.AIX)
 public class ManagedProducerRouteAddRemoveRegisterAlwaysTest extends ManagementTestSupport {
 
-    private static final int SERVICES = 14;
+    private static final int SERVICES = 15;
 
     @Override
     protected CamelContext createCamelContext() throws Exception {
diff --git a/core/camel-management/src/test/java/org/apache/camel/management/ManagedRouteAddRemoveTest.java b/core/camel-management/src/test/java/org/apache/camel/management/ManagedRouteAddRemoveTest.java
index abeda66326b..69c52be028b 100644
--- a/core/camel-management/src/test/java/org/apache/camel/management/ManagedRouteAddRemoveTest.java
+++ b/core/camel-management/src/test/java/org/apache/camel/management/ManagedRouteAddRemoveTest.java
@@ -40,7 +40,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
 @DisabledOnOs(OS.AIX)
 public class ManagedRouteAddRemoveTest extends ManagementTestSupport {
 
-    private static final int SERVICES = 14;
+    private static final int SERVICES = 15;
 
     @Override
     protected RouteBuilder createRouteBuilder() throws Exception {
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/AbstractExchange.java b/core/camel-support/src/main/java/org/apache/camel/support/AbstractExchange.java
index b0e4ddbcffc..362336c3292 100644
--- a/core/camel-support/src/main/java/org/apache/camel/support/AbstractExchange.java
+++ b/core/camel-support/src/main/java/org/apache/camel/support/AbstractExchange.java
@@ -53,7 +53,6 @@ abstract class AbstractExchange implements Exchange {
 
     protected final CamelContext context;
     protected Map<String, Object> properties; // create properties on-demand as we use internal properties mostly
-    protected Map<String, Object> variables;  // create variables on-demand
     protected Message in;
     protected Message out;
     protected Exception exception;
@@ -63,10 +62,10 @@ abstract class AbstractExchange implements Exchange {
     protected boolean rollbackOnly;
     protected boolean rollbackOnlyLast;
     protected Map<String, SafeCopyProperty> safeCopyProperties;
+    protected ExchangeVariableRepository variableRepository;
     private final ExtendedExchangeExtension privateExtension;
     private RedeliveryTraitPayload externalRedelivered = RedeliveryTraitPayload.UNDEFINED_REDELIVERY;
 
-    // TODO: variables ?
     protected AbstractExchange(CamelContext context, EnumMap<ExchangePropertyKey, Object> internalProperties,
                                Map<String, Object> properties) {
         this.context = context;
@@ -127,7 +126,11 @@ abstract class AbstractExchange implements Exchange {
         privateExtension.setStreamCacheDisabled(parent.getExchangeExtension().isStreamCacheDisabled());
 
         if (parent.hasVariables()) {
-            this.variables = safeCopyProperties(parent.variables);
+            if (this.variableRepository == null) {
+                this.variableRepository = new ExchangeVariableRepository();
+            }
+            this.variableRepository.setVariables(parent.getVariables());
+
         }
         if (parent.hasProperties()) {
             this.properties = safeCopyProperties(parent.properties);
@@ -391,8 +394,8 @@ abstract class AbstractExchange implements Exchange {
 
     @Override
     public Object getVariable(String name) {
-        if (variables != null) {
-            return variables.get(name);
+        if (variableRepository != null) {
+            return variableRepository.getVariable(name);
         }
         return null;
     }
@@ -411,24 +414,18 @@ abstract class AbstractExchange implements Exchange {
 
     @Override
     public void setVariable(String name, Object value) {
-        if (value != null) {
-            // avoid the NullPointException
-            if (variables == null) {
-                this.variables = new ConcurrentHashMap<>(8);
-            }
-            variables.put(name, value);
-        } else if (variables != null) {
-            // if the value is null, we just remove the key from the map
-            variables.remove(name);
+        if (variableRepository == null) {
+            variableRepository = new ExchangeVariableRepository();
         }
+        variableRepository.setVariable(name, value);
     }
 
     @Override
     public Object removeVariable(String name) {
-        if (!hasVariables()) {
-            return null;
+        if (variableRepository != null) {
+            return variableRepository.removeVariable(name);
         }
-        return variables.remove(name);
+        return null;
     }
 
     @Override
@@ -438,40 +435,37 @@ abstract class AbstractExchange implements Exchange {
 
     @Override
     public boolean removeVariables(String pattern, String... excludePatterns) {
+        if (variableRepository == null) {
+            return false;
+        }
+
         // special optimized
         if (excludePatterns == null && "*".equals(pattern)) {
-            if (variables != null) {
-                variables.clear();
-            }
+            variableRepository.clear();
             return true;
         }
 
         boolean matches = false;
 
         // store keys to be removed as we cannot loop and remove at the same time in implementations such as HashMap
-        if (variables != null) {
-            Set<String> toBeRemoved = null;
-            for (String key : variables.keySet()) {
+        if (variableRepository.hasVariables()) {
+            Set<String> toBeRemoved = new HashSet<>();
+            variableRepository.names().forEach(key -> {
                 if (PatternHelper.matchPattern(key, pattern)) {
-                    if (excludePatterns != null && PatternHelper.isExcludePatternMatch(key, excludePatterns)) {
-                        continue;
+                    boolean excluded = excludePatterns != null && PatternHelper.isExcludePatternMatch(key, excludePatterns);
+                    if (!excluded) {
+                        toBeRemoved.add(key);
                     }
-                    matches = true;
-                    if (toBeRemoved == null) {
-                        toBeRemoved = new HashSet<>();
-                    }
-                    toBeRemoved.add(key);
                 }
-            }
-
-            if (matches) {
-                if (toBeRemoved.size() == variables.size()) {
-                    // special optimization when all should be removed
-                    variables.clear();
-                } else {
-                    for (String key : toBeRemoved) {
-                        variables.remove(key);
-                    }
+            });
+
+            matches = !toBeRemoved.isEmpty();
+            if (toBeRemoved.size() == variableRepository.size()) {
+                // special optimization when all should be removed
+                variableRepository.clear();
+            } else {
+                for (String key : toBeRemoved) {
+                    variableRepository.removeVariable(key);
                 }
             }
         }
@@ -481,15 +475,19 @@ abstract class AbstractExchange implements Exchange {
 
     @Override
     public Map<String, Object> getVariables() {
-        if (variables == null) {
-            this.variables = new ConcurrentHashMap<>(8);
+        if (variableRepository == null) {
+            // force creating variables
+            variableRepository = new ExchangeVariableRepository();
         }
-        return variables;
+        return variableRepository.getVariables();
     }
 
     @Override
     public boolean hasVariables() {
-        return variables != null && !variables.isEmpty();
+        if (variableRepository != null) {
+            return variableRepository.hasVariables();
+        }
+        return false;
     }
 
     @Override
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/DefaultPooledExchange.java b/core/camel-support/src/main/java/org/apache/camel/support/DefaultPooledExchange.java
index 75fa2d70e42..620a7af2623 100644
--- a/core/camel-support/src/main/java/org/apache/camel/support/DefaultPooledExchange.java
+++ b/core/camel-support/src/main/java/org/apache/camel/support/DefaultPooledExchange.java
@@ -41,7 +41,6 @@ public final class DefaultPooledExchange extends AbstractExchange implements Poo
         super(context);
         this.originalPattern = getPattern();
         this.properties = new ConcurrentHashMap<>(8);
-
         this.clock = new ResetableClock();
     }
 
@@ -184,9 +183,7 @@ public final class DefaultPooledExchange extends AbstractExchange implements Poo
 
     public static DefaultPooledExchange newFromEndpoint(Endpoint fromEndpoint, ExchangePattern exchangePattern) {
         DefaultPooledExchange exchange = new DefaultPooledExchange(fromEndpoint.getCamelContext(), exchangePattern);
-
         exchange.getExchangeExtension().setFromEndpoint(fromEndpoint);
-
         return exchange;
     }
 }
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/ExchangeVariableRepository.java b/core/camel-support/src/main/java/org/apache/camel/support/ExchangeVariableRepository.java
new file mode 100644
index 00000000000..e422155282f
--- /dev/null
+++ b/core/camel-support/src/main/java/org/apache/camel/support/ExchangeVariableRepository.java
@@ -0,0 +1,87 @@
+/*
+ * 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.support;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.stream.Stream;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.NonManagedService;
+import org.apache.camel.spi.VariableRepository;
+import org.apache.camel.support.service.ServiceSupport;
+
+/**
+ * {@link VariableRepository} which is local per {@link Exchange} to hold request-scoped variables.
+ */
+class ExchangeVariableRepository extends ServiceSupport implements VariableRepository, NonManagedService {
+
+    private final Map<String, Object> variables = new ConcurrentHashMap<>(8);
+
+    @Override
+    public String getId() {
+        return "exchange";
+    }
+
+    @Override
+    public Object getVariable(String name) {
+        return variables.get(name);
+    }
+
+    @Override
+    public void setVariable(String name, Object value) {
+        if (value != null) {
+            // avoid the NullPointException
+            variables.put(name, value);
+        } else {
+            // if the value is null, we just remove the key from the map
+            variables.remove(name);
+        }
+    }
+
+    public boolean hasVariables() {
+        return !variables.isEmpty();
+    }
+
+    public int size() {
+        return variables.size();
+    }
+
+    public Stream<String> names() {
+        return variables.keySet().stream();
+    }
+
+    public Map<String, Object> getVariables() {
+        return variables;
+    }
+
+    public void setVariables(Map<String, Object> map) {
+        variables.putAll(map);
+    }
+
+    public void clear() {
+        variables.clear();
+    }
+
+    @Override
+    public Object removeVariable(String name) {
+        if (!hasVariables()) {
+            return null;
+        }
+        return variables.remove(name);
+    }
+}
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/ExtendedExchangeExtension.java b/core/camel-support/src/main/java/org/apache/camel/support/ExtendedExchangeExtension.java
index 42cb778526b..9f35486dd0b 100644
--- a/core/camel-support/src/main/java/org/apache/camel/support/ExtendedExchangeExtension.java
+++ b/core/camel-support/src/main/java/org/apache/camel/support/ExtendedExchangeExtension.java
@@ -328,14 +328,13 @@ public class ExtendedExchangeExtension implements ExchangeExtension {
         setRedeliveryExhausted(false);
         setErrorHandlerHandled(null);
         setStreamCacheDisabled(false);
+        this.exchange.variableRepository = null;
     }
 
     @Override
     public Exchange createCopyWithProperties(CamelContext context) {
         DefaultExchange answer = new DefaultExchange(context, exchange.internalProperties, exchange.properties);
-
         answer.setPattern(exchange.pattern);
-
         return answer;
     }
 }
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/GlobalVariableRepository.java b/core/camel-support/src/main/java/org/apache/camel/support/GlobalVariableRepository.java
new file mode 100644
index 00000000000..a026f759c8b
--- /dev/null
+++ b/core/camel-support/src/main/java/org/apache/camel/support/GlobalVariableRepository.java
@@ -0,0 +1,58 @@
+/*
+ * 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.support;
+
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import org.apache.camel.spi.VariableRepository;
+import org.apache.camel.support.service.ServiceSupport;
+
+/**
+ * Global {@link VariableRepository} which stores variables in-memory in a {@link Map}.
+ */
+public class GlobalVariableRepository extends ServiceSupport implements VariableRepository {
+
+    private final ConcurrentMap<String, Object> variables = new ConcurrentHashMap<>();
+
+    @Override
+    public String getId() {
+        return "global";
+    }
+
+    @Override
+    public Object getVariable(String name) {
+        return variables.get(name);
+    }
+
+    @Override
+    public void setVariable(String name, Object value) {
+        if (value != null) {
+            // avoid the NullPointException
+            variables.put(name, value);
+        } else {
+            // if the value is null, we just remove the key from the map
+            variables.remove(name);
+        }
+    }
+
+    @Override
+    public Object removeVariable(String name) {
+        return variables.remove(name);
+    }
+}
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/builder/ExpressionBuilder.java b/core/camel-support/src/main/java/org/apache/camel/support/builder/ExpressionBuilder.java
index b476a8c0e96..2d105ed02f8 100644
--- a/core/camel-support/src/main/java/org/apache/camel/support/builder/ExpressionBuilder.java
+++ b/core/camel-support/src/main/java/org/apache/camel/support/builder/ExpressionBuilder.java
@@ -294,7 +294,7 @@ public class ExpressionBuilder {
     }
 
     /**
-     * Returns an expression for variables
+     * Returns an expression for the {@link Exchange} variables
      *
      * @return an expression object which will return the variables
      */
diff --git a/core/camel-support/src/main/java/org/apache/camel/support/processor/DefaultExchangeFormatter.java b/core/camel-support/src/main/java/org/apache/camel/support/processor/DefaultExchangeFormatter.java
index ffd911aacf4..e9f919d5b45 100644
--- a/core/camel-support/src/main/java/org/apache/camel/support/processor/DefaultExchangeFormatter.java
+++ b/core/camel-support/src/main/java/org/apache/camel/support/processor/DefaultExchangeFormatter.java
@@ -185,7 +185,9 @@ public class DefaultExchangeFormatter implements ExchangeFormatter {
             if (multiline) {
                 sb.append(SEPARATOR);
             }
-            style(sb, "Variables").append(sortMap(filterHeaderAndProperties(exchange.getVariables())));
+            if (exchange.hasVariables()) {
+                style(sb, "Variables").append(sortMap(filterHeaderAndProperties(exchange.getVariables())));
+            }
         }
         if (showAll || showHeaders) {
             if (multiline) {
@@ -574,7 +576,9 @@ public class DefaultExchangeFormatter implements ExchangeFormatter {
 
     private static Map<String, Object> sortMap(Map<String, Object> map) {
         Map<String, Object> answer = new TreeMap<>(String.CASE_INSENSITIVE_ORDER);
-        answer.putAll(map);
+        if (map != null && !map.isEmpty()) {
+            answer.putAll(map);
+        }
         return answer;
     }
 
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 c5611e3ee33..8771410df5f 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
@@ -569,6 +569,9 @@ public final class StringHelper {
      * @return       the text after the token, or <tt>null</tt> if text does not contain the token
      */
     public static String after(String text, String after) {
+        if (text == null) {
+            return null;
+        }
         int pos = text.indexOf(after);
         if (pos == -1) {
             return null;
@@ -614,6 +617,9 @@ public final class StringHelper {
      * @return       the text after the token, or <tt>null</tt> if text does not contain the token
      */
     public static String afterLast(String text, String after) {
+        if (text == null) {
+            return null;
+        }
         int pos = text.lastIndexOf(after);
         if (pos == -1) {
             return null;
@@ -642,6 +648,9 @@ public final class StringHelper {
      * @return        the text before the token, or <tt>null</tt> if text does not contain the token
      */
     public static String before(String text, String before) {
+        if (text == null) {
+            return null;
+        }
         int pos = text.indexOf(before);
         return pos == -1 ? null : text.substring(0, pos);
     }
@@ -655,6 +664,9 @@ public final class StringHelper {
      * @return              the text before the token, or the supplied defaultValue if text does not contain the token
      */
     public static String before(String text, String before, String defaultValue) {
+        if (text == null) {
+            return defaultValue;
+        }
         int pos = text.indexOf(before);
         return pos == -1 ? defaultValue : text.substring(0, pos);
     }
@@ -668,6 +680,9 @@ public final class StringHelper {
      * @return              the text before the token, or the supplied defaultValue if text does not contain the token
      */
     public static String before(String text, char before, String defaultValue) {
+        if (text == null) {
+            return defaultValue;
+        }
         int pos = text.indexOf(before);
         return pos == -1 ? defaultValue : text.substring(0, pos);
     }
@@ -697,6 +712,9 @@ public final class StringHelper {
      * @return        the text before the token, or <tt>null</tt> if text does not contain the token
      */
     public static String beforeLast(String text, String before) {
+        if (text == null) {
+            return null;
+        }
         int pos = text.lastIndexOf(before);
         return pos == -1 ? null : text.substring(0, pos);
     }