You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by lb...@apache.org on 2020/09/21 17:54:44 UTC

[camel-k-runtime] branch master updated: Support endpoint DSL in YAML/JSON #366

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

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git


The following commit(s) were added to refs/heads/master by this push:
     new 5c44cb1  Support endpoint DSL in YAML/JSON #366
5c44cb1 is described below

commit 5c44cb1ba86db13aee11bc552a9cc274994b5ad5
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Jul 19 09:56:33 2020 +0200

    Support endpoint DSL in YAML/JSON #366
---
 .../k/loader/yaml/parser/EndpointStepParser.java   |    53 +
 .../camel/k/loader/yaml/parser/FromStepParser.java |    55 +-
 .../k/loader/yaml/parser/RouteStepParser.java      |    54 +-
 .../camel/k/loader/yaml/spi/HasEndpoint.java       |    27 +
 .../org/apache/camel/k/loader/yaml/spi/HasUri.java |    36 +
 .../apache/camel/k/loader/yaml/spi/StepParser.java |     4 +
 .../k/loader/yaml/support/StepParserSupport.java   |    16 +
 camel-k-loader-yaml/camel-k-loader-yaml/pom.xml    |    21 +
 .../src/generated/resources/camel-yaml-dsl.json    |     9 +-
 .../generated/resources/camel-yaml-endpoint.json   | 40044 +++++++++++++++++++
 .../k/loader/yaml/RouteWithEndpointTest.groovy     |    86 +
 .../k/loader/yaml/RoutesValidationTest.groovy      |     5 +-
 .../routes/RouteWithEndpointTest_from.yaml         |    26 +
 .../routes/RouteWithEndpointTest_route.yaml        |    27 +
 .../src/main/resources/application.properties      |     2 +-
 .../k/loader/yaml/quarkus/YamlLoaderTest.java      |    37 +-
 .../test/resources/routes_with_endpointdsl.yaml    |    31 +
 .../quarkus/deployment/DeploymentProcessor.java    |     4 +
 .../camel/k/annotation/yaml/JsonSchemaIgnore.java  |    27 +
 .../tooling/maven/GenerateYamlEndpointsSchema.java |   146 +
 .../maven/GenerateYamlLoaderSupportClasses.java    |    23 +-
 .../maven/GenerateYamlParserSupportClasses.java    |    70 +
 .../camel/k/tooling/maven/GenerateYamlSchema.java  |     5 +-
 .../camel/k/tooling/maven/GenerateYamlSupport.java |     8 +
 .../maven/processors/CatalogProcessor3x.java       |     1 +
 .../k/tooling/maven/support/ToolingSupport.java    |    58 +
 26 files changed, 40849 insertions(+), 26 deletions(-)

diff --git a/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/parser/EndpointStepParser.java b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/parser/EndpointStepParser.java
new file mode 100644
index 0000000..fcd477f
--- /dev/null
+++ b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/parser/EndpointStepParser.java
@@ -0,0 +1,53 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import com.fasterxml.jackson.databind.node.ObjectNode;
+import org.apache.camel.k.loader.yaml.spi.ProcessorStepParser;
+import org.apache.camel.k.loader.yaml.support.StepParserSupport;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.ToDefinition;
+import org.apache.camel.util.StringHelper;
+
+public class EndpointStepParser implements ProcessorStepParser {
+    private final String scheme;
+
+    public EndpointStepParser(String scheme) {
+        this.scheme = scheme;
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        final ObjectNode node = context.node(ObjectNode.class);
+        final Map<String, Object> parameters = new HashMap<>();
+
+        node.fields().forEachRemaining(entry -> {
+            parameters.put(
+                StringHelper.dashToCamelCase(entry.getKey()),
+                entry.getValue().asText()
+            );
+        });
+
+        return new ToDefinition(
+            StepParserSupport.createEndpointUri(context.getCamelContext(), this.scheme, parameters)
+        );
+    }
+}
+
diff --git a/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/parser/FromStepParser.java b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/parser/FromStepParser.java
index b4424db..9d05fb0 100644
--- a/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/parser/FromStepParser.java
+++ b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/parser/FromStepParser.java
@@ -19,40 +19,81 @@ package org.apache.camel.k.loader.yaml.parser;
 import java.util.List;
 import java.util.Map;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.camel.k.annotation.yaml.YAMLNodeDefinition;
 import org.apache.camel.k.annotation.yaml.YAMLStepParser;
 import org.apache.camel.k.loader.yaml.model.Step;
 import org.apache.camel.k.loader.yaml.spi.StartStepParser;
 import org.apache.camel.k.loader.yaml.support.StepParserSupport;
-import org.apache.camel.model.RouteDefinition;
 
 @YAMLStepParser(id = "from", definition = FromStepParser.Definition.class)
 public class FromStepParser implements StartStepParser {
     @Override
     public Object process(Context context) {
         final Definition definition = context.node(Definition.class);
-        final String uri = StepParserSupport.createEndpointUri(definition.uri, definition.parameters);
-        final RouteDefinition route = context.builder().from(uri);
+        if (definition.uri == null && definition.scheme == null) {
+            throw new IllegalArgumentException("Either uri or scheme must be set");
+        }
+
+        String uri = definition.uri != null
+            ? StepParserSupport.createEndpointUri(definition.uri, definition.parameters)
+            : StepParserSupport.createEndpointUri(context.getCamelContext(), definition.scheme, definition.parameters);
 
         // as this is a start converter, steps are mandatory
         StepParserSupport.notNull(definition.steps, "steps");
 
         return StepParserSupport.convertSteps(
             context,
-            route,
+            context.builder().from(uri),
             definition.steps
         );
     }
 
     @YAMLNodeDefinition
-    public static final class Definition {
-        @JsonProperty(required = true)
+    public static final class Definition implements HasEndpointConsumer {
+        public String scheme;
         public String uri;
-        @JsonProperty
         public Map<String, Object> parameters;
+
         @JsonProperty(required = true)
         public List<Step> steps;
+
+        @JsonIgnore
+        @Override
+        public void setEndpointScheme(String scheme) {
+            this.scheme = scheme;
+        }
+
+        @JsonIgnore
+        @Override
+        public String getEndpointScheme() {
+            return this.scheme ;
+        }
+
+        @JsonProperty(required = true)
+        @Override
+        public void setUri(String uri) {
+            this.uri = uri;
+        }
+
+        @JsonProperty
+        @Override
+        public String getUri() {
+            return this.uri;
+        }
+
+        @JsonProperty
+        @Override
+        public void setParameters(Map<String, Object> parameters) {
+            this.parameters = parameters;
+        }
+
+        @JsonProperty
+        @Override
+        public Map<String, Object> getParameters() {
+            return this.parameters;
+        }
     }
 }
 
diff --git a/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/parser/RouteStepParser.java b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/parser/RouteStepParser.java
index 602f36c..a13f02c 100644
--- a/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/parser/RouteStepParser.java
+++ b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/parser/RouteStepParser.java
@@ -19,6 +19,7 @@ package org.apache.camel.k.loader.yaml.parser;
 import java.util.List;
 import java.util.Map;
 
+import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import org.apache.camel.k.annotation.yaml.YAMLNodeDefinition;
 import org.apache.camel.k.annotation.yaml.YAMLStepParser;
@@ -33,8 +34,15 @@ public class RouteStepParser implements StartStepParser {
     @Override
     public Object process(Context context) {
         final Definition definition = context.node(Definition.class);
-        final String uri = StepParserSupport.createEndpointUri(definition.from.uri, definition.from.parameters);
-        final RouteDefinition route = context.builder().from(uri);
+        if (definition.from.uri == null && definition.from.scheme == null) {
+            throw new IllegalArgumentException("Either uri or scheme must be set");
+        }
+
+        String uri = definition.from.uri != null
+            ? StepParserSupport.createEndpointUri(definition.from.uri, definition.from.parameters)
+            : StepParserSupport.createEndpointUri(context.getCamelContext(), definition.from.scheme, definition.from.parameters);
+
+        RouteDefinition route = context.builder().from(uri);
 
         ObjectHelper.ifNotEmpty(definition.id, route::routeId);
         ObjectHelper.ifNotEmpty(definition.group, route::routeGroup);
@@ -62,11 +70,10 @@ public class RouteStepParser implements StartStepParser {
     }
 
     @YAMLNodeDefinition
-    public static final class From {
-        @JsonProperty
+    public static final class From implements HasEndpointConsumer {
         public String uri;
-        @JsonProperty
         public Map<String, Object> parameters;
+        public String scheme;
 
         public From() {
         }
@@ -74,6 +81,43 @@ public class RouteStepParser implements StartStepParser {
         public From(String uri) {
             this.uri = uri;
         }
+
+        @JsonIgnore
+        @Override
+        public void setEndpointScheme(String scheme) {
+            this.scheme = scheme;
+        }
+
+        @JsonIgnore
+        @Override
+        public String getEndpointScheme() {
+            return null;
+        }
+
+        @JsonProperty(required = true)
+        @Override
+        public void setUri(String uri) {
+            this.uri = uri;
+        }
+
+        @JsonProperty
+        @Override
+        public String getUri() {
+            return this.uri;
+        }
+
+        @JsonProperty
+        @Override
+        public void setParameters(Map<String, Object> parameters) {
+            this.parameters = parameters;
+
+        }
+
+        @JsonProperty
+        @Override
+        public Map<String, Object> getParameters() {
+            return this.parameters;
+        }
     }
 }
 
diff --git a/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/spi/HasEndpoint.java b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/spi/HasEndpoint.java
new file mode 100644
index 0000000..aa3d7a9
--- /dev/null
+++ b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/spi/HasEndpoint.java
@@ -0,0 +1,27 @@
+/*
+ * 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.k.loader.yaml.spi;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+
+public interface HasEndpoint extends HasUri {
+    @JsonIgnore
+    void setEndpointScheme(String scheme);
+
+    @JsonIgnore
+    String getEndpointScheme();
+}
diff --git a/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/spi/HasUri.java b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/spi/HasUri.java
new file mode 100644
index 0000000..c1561a8
--- /dev/null
+++ b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/spi/HasUri.java
@@ -0,0 +1,36 @@
+/*
+ * 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.k.loader.yaml.spi;
+
+import java.util.Map;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public interface HasUri {
+    @JsonProperty
+    void setUri(String uri);
+
+    @JsonProperty
+    String getUri();
+
+    @JsonIgnore
+    void setParameters(Map<String, Object> parameters);
+
+    @JsonIgnore
+    Map<String, Object> getParameters();
+}
diff --git a/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/spi/StepParser.java b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/spi/StepParser.java
index 5009dae..4fdaf62 100644
--- a/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/spi/StepParser.java
+++ b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/spi/StepParser.java
@@ -56,6 +56,10 @@ public interface StepParser {
             return builder.getContext();
         }
 
+        public <T extends CamelContext> T getCamelContext(Class<T> type) {
+            return builder.getContext().adapt(type);
+        }
+
         public ProcessorDefinition<?> processor() {
             return this.processor;
         }
diff --git a/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/support/StepParserSupport.java b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/support/StepParserSupport.java
index 388984b..1889c18 100644
--- a/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/support/StepParserSupport.java
+++ b/camel-k-loader-yaml/camel-k-loader-yaml-common/src/main/java/org/apache/camel/k/loader/yaml/support/StepParserSupport.java
@@ -16,10 +16,15 @@
  */
 package org.apache.camel.k.loader.yaml.support;
 
+import java.net.URISyntaxException;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.stream.Collectors;
 
+import org.apache.camel.CamelContext;
+import org.apache.camel.ExtendedCamelContext;
 import org.apache.camel.k.loader.yaml.model.Step;
 import org.apache.camel.k.loader.yaml.spi.ProcessorStepParser;
 import org.apache.camel.k.loader.yaml.spi.StepParser;
@@ -82,4 +87,15 @@ public final class StepParserSupport {
 
         return answer;
     }
+
+    public static String createEndpointUri(CamelContext context, String scheme, Map<String, Object> parameters) {
+        try {
+            Map<String, String> params = new HashMap<>();
+            parameters.forEach((k, v) -> params.put(k, Objects.toString(v)));
+
+            return context.adapt(ExtendedCamelContext.class).getRuntimeCamelCatalog().asEndpointUri(scheme, params, false);
+        } catch (URISyntaxException e) {
+            throw new IllegalArgumentException(e);
+        }
+    }
 }
diff --git a/camel-k-loader-yaml/camel-k-loader-yaml/pom.xml b/camel-k-loader-yaml/camel-k-loader-yaml/pom.xml
index 738fe6f..58a43e8 100644
--- a/camel-k-loader-yaml/camel-k-loader-yaml/pom.xml
+++ b/camel-k-loader-yaml/camel-k-loader-yaml/pom.xml
@@ -36,6 +36,11 @@
         <!-- ****************************** -->
 
         <dependency>
+            <groupId>org.apache.camel</groupId>
+            <artifactId>camel-core-catalog</artifactId>
+        </dependency>
+
+        <dependency>
             <groupId>org.apache.camel.k</groupId>
             <artifactId>camel-k-runtime-core</artifactId>
         </dependency>
@@ -101,6 +106,11 @@
             <scope>test</scope>
         </dependency>
         <dependency>
+            <groupId>org.apache.camel</groupId>
+            <artifactId>camel-telegram</artifactId>
+            <scope>test</scope>
+        </dependency>
+        <dependency>
             <groupId>com.github.java-json-tools</groupId>
             <artifactId>json-schema-validator</artifactId>
             <version>${json-schema-validator-version}</version>
@@ -172,6 +182,7 @@
                 <version>${project.version}</version>
                 <executions>
                     <execution>
+                        <id>generate-yaml-schema</id>
                         <phase>generate-sources</phase>
                         <goals>
                             <goal>generate-yaml-schema</goal>
@@ -195,6 +206,16 @@
                             </bannedDefinitions>
                         </configuration>
                     </execution>
+                    <execution>
+                        <id>generate-yaml-endpoint-schema</id>
+                        <phase>generate-sources</phase>
+                        <goals>
+                            <goal>generate-yaml-endpoints-schema</goal>
+                        </goals>
+                        <configuration>
+                            <outputFile>${project.basedir}/src/generated/resources/camel-yaml-endpoint.json</outputFile>
+                        </configuration>
+                    </execution>
                 </executions>
             </plugin>
             <plugin>
diff --git a/camel-k-loader-yaml/camel-k-loader-yaml/src/generated/resources/camel-yaml-dsl.json b/camel-k-loader-yaml/camel-k-loader-yaml/src/generated/resources/camel-yaml-dsl.json
index 2bf79bf..4d18bdd 100644
--- a/camel-k-loader-yaml/camel-k-loader-yaml/src/generated/resources/camel-yaml-dsl.json
+++ b/camel-k-loader-yaml/camel-k-loader-yaml/src/generated/resources/camel-yaml-dsl.json
@@ -2327,15 +2327,15 @@
       "org.apache.camel.k.loader.yaml.parser.FromStepParser$Definition" : {
         "type" : "object",
         "properties" : {
-          "parameters" : {
-            "type" : "object"
-          },
           "steps" : {
             "type" : "array",
             "items" : {
               "$ref" : "#/items/definitions/step"
             }
           },
+          "parameters" : {
+            "type" : "object"
+          },
           "uri" : {
             "type" : "string"
           }
@@ -2897,7 +2897,8 @@
             "uri" : {
               "type" : "string"
             }
-          }
+          },
+          "required" : [ "uri" ]
         } ]
       },
       "org.apache.camel.k.loader.yaml.parser.RoutingSlipStepParser$Definition" : {
diff --git a/camel-k-loader-yaml/camel-k-loader-yaml/src/generated/resources/camel-yaml-endpoint.json b/camel-k-loader-yaml/camel-k-loader-yaml/src/generated/resources/camel-yaml-endpoint.json
new file mode 100644
index 0000000..a5baa42
--- /dev/null
+++ b/camel-k-loader-yaml/camel-k-loader-yaml/src/generated/resources/camel-yaml-endpoint.json
@@ -0,0 +1,40044 @@
+{
+  "$schema" : "http://json-schema.org/draft-04/schema#",
+  "type" : "object",
+  "definitions" : {
+    "activemq" : {
+      "type" : "object",
+      "required" : [ "destinationName" ],
+      "properties" : {
+        "destination-name" : {
+          "description" : "Name of the queue or topic to use as destination",
+          "type" : "string"
+        },
+        "destination-type" : {
+          "description" : "The kind of destination to use",
+          "default" : "queue",
+          "enum" : [ "queue", "topic", "temp-queue", "temp-topic" ],
+          "type" : "string"
+        },
+        "accept-messages-while-stopping" : {
+          "description" : "Specifies whether the consumer accept messages while it is stopping. You may consider enabling this option, if you start and stop JMS routes at runtime, while there are still messages enqueued on the queue. If this option is false, and you stop the JMS route, then messages may be rejected, and the JMS broker would have to attempt redeliveries, which yet again may be rejected, and eventually the message may be moved at a dead letter queue on the JMS broker. To a [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "acknowledgement-mode-name" : {
+          "description" : "The JMS acknowledgement name, which is one of: SESSION_TRANSACTED, CLIENT_ACKNOWLEDGE, AUTO_ACKNOWLEDGE, DUPS_OK_ACKNOWLEDGE",
+          "default" : "AUTO_ACKNOWLEDGE",
+          "enum" : [ "SESSION_TRANSACTED", "CLIENT_ACKNOWLEDGE", "AUTO_ACKNOWLEDGE", "DUPS_OK_ACKNOWLEDGE" ],
+          "type" : "string"
+        },
+        "allow-additional-headers" : {
+          "description" : "This option is used to allow additional headers which may have values that are invalid according to JMS specification. For example some message systems such as WMQ do this with header names using prefix JMS_IBM_MQMD_ containing values with byte array or other invalid types. You can specify multiple header names separated by comma, and use as suffix for wildcard matching.",
+          "type" : "string"
+        },
+        "allow-null-body" : {
+          "description" : "Whether to allow sending messages with no body. If this option is false and the message body is null, then an JMSException is thrown.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "allow-reply-manager-quick-stop" : {
+          "description" : "Whether the DefaultMessageListenerContainer used in the reply managers for request-reply messaging allow the DefaultMessageListenerContainer.runningAllowed flag to quick stop in case JmsConfiguration#isAcceptMessagesWhileStopping is enabled, and org.apache.camel.CamelContext is currently being stopped. This quick stop ability is enabled by default in the regular JMS consumers but to enable for reply managers you must enable this flag.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "allow-serialized-headers" : {
+          "description" : "Controls whether or not to include serialized headers. Applies only when transferExchange is true. This requires that the objects are serializable. Camel will exclude any non-serializable objects and log it at WARN level.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "always-copy-message" : {
+          "description" : "If true, Camel will always make a JMS message copy of the message when it is passed to the producer for sending. Copying the message is needed in some situations, such as when a replyToDestinationSelectorName is set (incidentally, Camel will set the alwaysCopyMessage option to true, if a replyToDestinationSelectorName is set)",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "artemis-streaming-enabled" : {
+          "description" : "Whether optimizing for Apache Artemis streaming mode.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "async-consumer" : {
+          "description" : "Whether the JmsConsumer processes the Exchange asynchronously. If enabled then the JmsConsumer may pickup the next message from the JMS queue, while the previous message is being processed asynchronously (by the Asynchronous Routing Engine). This means that messages may be processed not 100% strictly in order. If disabled (as default) then the Exchange is fully processed before the JmsConsumer will pickup the next message from the JMS queue. Note if transacted  [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "async-start-listener" : {
+          "description" : "Whether to startup the JmsConsumer message listener asynchronously, when starting a route. For example if a JmsConsumer cannot get a connection to a remote JMS broker, then it may block while retrying and/or failover. This will cause Camel to block while starting routes. By setting this option to true, you will let routes startup, while the JmsConsumer connects to the JMS broker using a dedicated thread in asynchronous mode. If this option is used, then beware  [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "async-stop-listener" : {
+          "description" : "Whether to stop the JmsConsumer message listener asynchronously, when stopping a route.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "auto-startup" : {
+          "description" : "Specifies whether the consumer container should auto-startup.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "cache-level" : {
+          "description" : "Sets the cache level by ID for the underlying JMS resources. See cacheLevelName option for more details.",
+          "type" : "integer"
+        },
+        "cache-level-name" : {
+          "description" : "Sets the cache level by name for the underlying JMS resources. Possible values are: CACHE_AUTO, CACHE_CONNECTION, CACHE_CONSUMER, CACHE_NONE, and CACHE_SESSION. The default setting is CACHE_AUTO. See the Spring documentation and Transactions Cache Levels for more information.",
+          "default" : "CACHE_AUTO",
+          "enum" : [ "CACHE_AUTO", "CACHE_CONNECTION", "CACHE_CONSUMER", "CACHE_NONE", "CACHE_SESSION" ],
+          "type" : "string"
+        },
+        "client-id" : {
+          "description" : "Sets the JMS client ID to use. Note that this value, if specified, must be unique and can only be used by a single JMS connection instance. It is typically only required for durable topic subscriptions. If using Apache ActiveMQ you may prefer to use Virtual Topics instead.",
+          "type" : "string"
+        },
+        "concurrent-consumers" : {
+          "description" : "Specifies the default number of concurrent consumers when consuming from JMS (not for request/reply over JMS). See also the maxMessagesPerTask option to control dynamic scaling up/down of threads. When doing request/reply over JMS then the option replyToConcurrentConsumers is used to control number of concurrent consumers on the reply message listener.",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "connection-factory" : {
+          "description" : "The connection factory to be use. A connection factory must be configured either on the component or endpoint.",
+          "type" : "string"
+        },
+        "consumer-type" : {
+          "description" : "The consumer type to use, which can be one of: Simple, Default, or Custom. The consumer type determines which Spring JMS listener to use. Default will use org.springframework.jms.listener.DefaultMessageListenerContainer, Simple will use org.springframework.jms.listener.SimpleMessageListenerContainer. When Custom is specified, the MessageListenerContainerFactory defined by the messageListenerContainerFactory option will determine what org.springframework.jms.lis [...]
+          "default" : "Default",
+          "enum" : [ "Simple", "Default", "Custom" ],
+          "type" : "string"
+        },
+        "correlation-property" : {
+          "description" : "When using InOut exchange pattern use this JMS property instead of JMSCorrelationID JMS property to correlate messages. If set messages will be correlated solely on the value of this property JMSCorrelationID property will be ignored and not set by Camel.",
+          "type" : "string"
+        },
+        "default-task-executor-type" : {
+          "description" : "Specifies what default TaskExecutor type to use in the DefaultMessageListenerContainer, for both consumer endpoints and the ReplyTo consumer of producer endpoints. Possible values: SimpleAsync (uses Spring's SimpleAsyncTaskExecutor) or ThreadPool (uses Spring's ThreadPoolTaskExecutor with optimal values - cached threadpool-like). If not set, it defaults to the previous behaviour, which uses a cached thread pool for consumer endpoints and SimpleAsync for reply c [...]
+          "enum" : [ "ThreadPool", "SimpleAsync" ],
+          "type" : "string"
+        },
+        "delivery-delay" : {
+          "description" : "Sets delivery delay to use for send calls for JMS. This option requires JMS 2.0 compliant broker.",
+          "default" : "-1",
+          "type" : "integer"
+        },
+        "delivery-mode" : {
+          "description" : "Specifies the delivery mode to be used. Possibles values are those defined by javax.jms.DeliveryMode. NON_PERSISTENT = 1 and PERSISTENT = 2.",
+          "enum" : [ "1", "2" ],
+          "type" : "integer"
+        },
+        "delivery-persistent" : {
+          "description" : "Specifies whether persistent delivery is used by default.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "destination-resolver" : {
+          "description" : "A pluggable org.springframework.jms.support.destination.DestinationResolver that allows you to use your own resolver (for example, to lookup the real destination in a JNDI registry).",
+          "type" : "string"
+        },
+        "disable-reply-to" : {
+          "description" : "Specifies whether Camel ignores the JMSReplyTo header in messages. If true, Camel does not send a reply back to the destination specified in the JMSReplyTo header. You can use this option if you want Camel to consume from a route and you do not want Camel to automatically send back a reply message because another component in your code handles the reply message. You can also use this option if you want to use Camel as a proxy between different message brokers a [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "disable-time-to-live" : {
+          "description" : "Use this option to force disabling time to live. For example when you do request/reply over JMS, then Camel will by default use the requestTimeout value as time to live on the message being sent. The problem is that the sender and receiver systems have to have their clocks synchronized, so they are in sync. This is not always so easy to archive. So you can use disableTimeToLive=true to not set a time to live value on the sent message. Then the message will not  [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "durable-subscription-name" : {
+          "description" : "The durable subscriber name for specifying durable topic subscriptions. The clientId option must be configured as well.",
+          "type" : "string"
+        },
+        "eager-loading-of-properties" : {
+          "description" : "Enables eager loading of JMS properties and payload as soon as a message is loaded which generally is inefficient as the JMS properties may not be required but sometimes can catch early any issues with the underlying JMS provider and the use of JMS properties. See also the option eagerPoisonBody.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "eager-poison-body" : {
+          "description" : "If eagerLoadingOfProperties is enabled and the JMS message payload (JMS body or JMS properties) is poison (cannot be read/mapped), then set this text as the message body instead so the message can be processed (the cause of the poison are already stored as exception on the Exchange). This can be turned off by setting eagerPoisonBody=false. See also the option eagerLoadingOfProperties.",
+          "default" : "Poison JMS message due to ${exception.message}",
+          "type" : "string"
+        },
+        "error-handler" : {
+          "description" : "Specifies a org.springframework.util.ErrorHandler to be invoked in case of any uncaught exceptions thrown while processing a Message. By default these exceptions will be logged at the WARN level, if no errorHandler has been configured. You can configure logging level and whether stack traces should be logged using errorHandlerLoggingLevel and errorHandlerLogStackTrace options. This makes it much easier to configure, than having to code a custom errorHandler.",
+          "type" : "string"
+        },
+        "error-handler-log-stack-trace" : {
+          "description" : "Allows to control whether stacktraces should be logged or not, by the default errorHandler.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "error-handler-logging-level" : {
+          "description" : "Allows to configure the default errorHandler logging level for logging uncaught exceptions.",
+          "default" : "WARN",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exception-listener" : {
+          "description" : "Specifies the JMS Exception Listener that is to be notified of any underlying JMS exceptions.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "explicit-qos-enabled" : {
+          "description" : "Set if the deliveryMode, priority or timeToLive qualities of service should be used when sending messages. This option is based on Spring's JmsTemplate. The deliveryMode, priority and timeToLive options are applied to the current endpoint. This contrasts with the preserveMessageQos option, which operates at message granularity, reading QoS properties exclusively from the Camel In message headers.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "expose-listener-session" : {
+          "description" : "Specifies whether the listener session should be exposed when consuming messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "force-send-original-message" : {
+          "description" : "When using mapJmsMessage=false Camel will create a new JMS message to send to a new JMS destination if you touch the headers (get or set) during the route. Set this option to true to force Camel to send the original JMS message that was received.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "format-date-headers-to-iso8601" : {
+          "description" : "Sets whether JMS date properties should be formatted according to the ISO 8601 standard.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "header-filter-strategy" : {
+          "description" : "To use a custom HeaderFilterStrategy to filter header to and from Camel message.",
+          "type" : "string"
+        },
+        "idle-consumer-limit" : {
+          "description" : "Specify the limit for the number of consumers that are allowed to be idle at any given time.",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "idle-task-execution-limit" : {
+          "description" : "Specifies the limit for idle executions of a receive task, not having received any message within its execution. If this limit is reached, the task will shut down and leave receiving to other executing tasks (in the case of dynamic scheduling; see the maxConcurrentConsumers setting). There is additional doc available from Spring.",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "include-all-jmsx-properties" : {
+          "description" : "Whether to include all JMSXxxx properties when mapping from JMS to Camel Message. Setting this to true will include properties such as JMSXAppID, and JMSXUserID etc. Note: If you are using a custom headerFilterStrategy then this option does not apply.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "include-sent-jms-message-id" : {
+          "description" : "Only applicable when sending to JMS destination using InOnly (eg fire and forget). Enabling this option will enrich the Camel Exchange with the actual JMSMessageID that was used by the JMS client when the message was sent to the JMS destination.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "jms-key-format-strategy" : {
+          "description" : "Pluggable strategy for encoding and decoding JMS keys so they can be compliant with the JMS specification. Camel provides two implementations out of the box: default and passthrough. The default strategy will safely marshal dots and hyphens (. and -). The passthrough strategy leaves the key as is. Can be used for JMS brokers which do not care whether JMS header keys contain illegal characters. You can provide your own implementation of the org.apache.camel.comp [...]
+          "enum" : [ "default", "passthrough" ],
+          "type" : "string"
+        },
+        "jms-message-type" : {
+          "description" : "Allows you to force the use of a specific javax.jms.Message implementation for sending JMS messages. Possible values are: Bytes, Map, Object, Stream, Text. By default, Camel would determine which JMS message type to use from the In body type. This option allows you to specify it.",
+          "enum" : [ "Bytes", "Map", "Object", "Stream", "Text" ],
+          "type" : "string"
+        },
+        "lazy-create-transaction-manager" : {
+          "description" : "If true, Camel will create a JmsTransactionManager, if there is no transactionManager injected when option transacted=true.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "map-jms-message" : {
+          "description" : "Specifies whether Camel should auto map the received JMS message to a suited payload type, such as javax.jms.TextMessage to a String etc.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "max-concurrent-consumers" : {
+          "description" : "Specifies the maximum number of concurrent consumers when consuming from JMS (not for request/reply over JMS). See also the maxMessagesPerTask option to control dynamic scaling up/down of threads. When doing request/reply over JMS then the option replyToMaxConcurrentConsumers is used to control number of concurrent consumers on the reply message listener.",
+          "type" : "integer"
+        },
+        "max-messages-per-task" : {
+          "description" : "The number of messages per task. -1 is unlimited. If you use a range for concurrent consumers (eg min max), then this option can be used to set a value to eg 100 to control how fast the consumers will shrink when less work is required.",
+          "default" : "-1",
+          "type" : "integer"
+        },
+        "message-converter" : {
+          "description" : "To use a custom Spring org.springframework.jms.support.converter.MessageConverter so you can be in control how to map to/from a javax.jms.Message.",
+          "type" : "string"
+        },
+        "message-created-strategy" : {
+          "description" : "To use the given MessageCreatedStrategy which are invoked when Camel creates new instances of javax.jms.Message objects when Camel is sending a JMS message.",
+          "type" : "string"
+        },
+        "message-id-enabled" : {
+          "description" : "When sending, specifies whether message IDs should be added. This is just an hint to the JMS broker. If the JMS provider accepts this hint, these messages must have the message ID set to null; if the provider ignores the hint, the message ID must be set to its normal unique value.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "message-listener-container-factory" : {
+          "description" : "Registry ID of the MessageListenerContainerFactory used to determine what org.springframework.jms.listener.AbstractMessageListenerContainer to use to consume messages. Setting this will automatically set consumerType to Custom.",
+          "type" : "string"
+        },
+        "message-timestamp-enabled" : {
+          "description" : "Specifies whether timestamps should be enabled by default on sending messages. This is just an hint to the JMS broker. If the JMS provider accepts this hint, these messages must have the timestamp set to zero; if the provider ignores the hint the timestamp must be set to its normal value.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "password" : {
+          "description" : "Password to use with the ConnectionFactory. You can also configure username/password directly on the ConnectionFactory.",
+          "type" : "string"
+        },
+        "preserve-message-qos" : {
+          "description" : "Set to true, if you want to send message using the QoS settings specified on the message, instead of the QoS settings on the JMS endpoint. The following three headers are considered JMSPriority, JMSDeliveryMode, and JMSExpiration. You can provide all or only some of them. If not provided, Camel will fall back to use the values from the endpoint instead. So, when using this option, the headers override the values from the endpoint. The explicitQosEnabled option, [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "priority" : {
+          "description" : "Values greater than 1 specify the message priority when sending (where 0 is the lowest priority and 9 is the highest). The explicitQosEnabled option must also be enabled in order for this option to have any effect.",
+          "default" : "4",
+          "enum" : [ "1", "2", "3", "4", "5", "6", "7", "8", "9" ],
+          "type" : "integer"
+        },
+        "pub-sub-no-local" : {
+          "description" : "Specifies whether to inhibit the delivery of messages published by its own connection.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "receive-timeout" : {
+          "description" : "The timeout for receiving messages (in milliseconds).",
+          "default" : "1000",
+          "type" : "string"
+        },
+        "recovery-interval" : {
+          "description" : "Specifies the interval between recovery attempts, i.e. when a connection is being refreshed, in milliseconds. The default is 5000 ms, that is, 5 seconds.",
+          "default" : "5000",
+          "type" : "string"
+        },
+        "reply-to" : {
+          "description" : "Provides an explicit ReplyTo destination, which overrides any incoming value of Message.getJMSReplyTo().",
+          "type" : "string"
+        },
+        "reply-to-cache-level-name" : {
+          "description" : "Sets the cache level by name for the reply consumer when doing request/reply over JMS. This option only applies when using fixed reply queues (not temporary). Camel will by default use: CACHE_CONSUMER for exclusive or shared w/ replyToSelectorName. And CACHE_SESSION for shared without replyToSelectorName. Some JMS brokers such as IBM WebSphere may require to set the replyToCacheLevelName=CACHE_NONE to work. Note: If using temporary queues then CACHE_NONE is not [...]
+          "enum" : [ "CACHE_AUTO", "CACHE_CONNECTION", "CACHE_CONSUMER", "CACHE_NONE", "CACHE_SESSION" ],
+          "type" : "string"
+        },
+        "reply-to-concurrent-consumers" : {
+          "description" : "Specifies the default number of concurrent consumers when doing request/reply over JMS. See also the maxMessagesPerTask option to control dynamic scaling up/down of threads.",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "reply-to-delivery-persistent" : {
+          "description" : "Specifies whether to use persistent delivery by default for replies.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "reply-to-destination-selector-name" : {
+          "description" : "Sets the JMS Selector using the fixed name to be used so you can filter out your own replies from the others when using a shared queue (that is, if you are not using a temporary reply queue).",
+          "type" : "string"
+        },
+        "reply-to-max-concurrent-consumers" : {
+          "description" : "Specifies the maximum number of concurrent consumers when using request/reply over JMS. See also the maxMessagesPerTask option to control dynamic scaling up/down of threads.",
+          "type" : "integer"
+        },
+        "reply-to-on-timeout-max-concurrent-consumers" : {
+          "description" : "Specifies the maximum number of concurrent consumers for continue routing when timeout occurred when using request/reply over JMS.",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "reply-to-override" : {
+          "description" : "Provides an explicit ReplyTo destination in the JMS message, which overrides the setting of replyTo. It is useful if you want to forward the message to a remote Queue and receive the reply message from the ReplyTo destination.",
+          "type" : "string"
+        },
+        "reply-to-same-destination-allowed" : {
+          "description" : "Whether a JMS consumer is allowed to send a reply message to the same destination that the consumer is using to consume from. This prevents an endless loop by consuming and sending back the same message to itself.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "reply-to-type" : {
+          "description" : "Allows for explicitly specifying which kind of strategy to use for replyTo queues when doing request/reply over JMS. Possible values are: Temporary, Shared, or Exclusive. By default Camel will use temporary queues. However if replyTo has been configured, then Shared is used by default. This option allows you to use exclusive queues instead of shared ones. See Camel JMS documentation for more details, and especially the notes about the implications if running in [...]
+          "enum" : [ "Temporary", "Shared", "Exclusive" ],
+          "type" : "string"
+        },
+        "request-timeout" : {
+          "description" : "The timeout for waiting for a reply when using the InOut Exchange Pattern (in milliseconds). The default is 20 seconds. You can include the header CamelJmsRequestTimeout to override this endpoint configured timeout value, and thus have per message individual timeout values. See also the requestTimeoutCheckerInterval option.",
+          "default" : "20000",
+          "type" : "string"
+        },
+        "request-timeout-checker-interval" : {
+          "description" : "Configures how often Camel should check for timed out Exchanges when doing request/reply over JMS. By default Camel checks once per second. But if you must react faster when a timeout occurs, then you can lower this interval, to check more frequently. The timeout is determined by the option requestTimeout.",
+          "default" : "1000",
+          "type" : "string"
+        },
+        "selector" : {
+          "description" : "Sets the JMS selector to use",
+          "type" : "string"
+        },
+        "stream-message-type-enabled" : {
+          "description" : "Sets whether StreamMessage type is enabled or not. Message payloads of streaming kind such as files, InputStream, etc will either by sent as BytesMessage or StreamMessage. This option controls which kind will be used. By default BytesMessage is used which enforces the entire message payload to be read into memory. By enabling this option the message payload is read into memory in chunks and each chunk is then written to the StreamMessage until no more data.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "subscription-durable" : {
+          "description" : "Set whether to make the subscription durable. The durable subscription name to be used can be specified through the subscriptionName property. Default is false. Set this to true to register a durable subscription, typically in combination with a subscriptionName value (unless your message listener class name is good enough as subscription name). Only makes sense when listening to a topic (pub-sub domain), therefore this method switches the pubSubDomain flag as well.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "subscription-name" : {
+          "description" : "Set the name of a subscription to create. To be applied in case of a topic (pub-sub domain) with a shared or durable subscription. The subscription name needs to be unique within this client's JMS client id. Default is the class name of the specified message listener. Note: Only 1 concurrent consumer (which is the default of this message listener container) is allowed for each subscription, except for a shared subscription (which requires JMS 2.0).",
+          "type" : "string"
+        },
+        "subscription-shared" : {
+          "description" : "Set whether to make the subscription shared. The shared subscription name to be used can be specified through the subscriptionName property. Default is false. Set this to true to register a shared subscription, typically in combination with a subscriptionName value (unless your message listener class name is good enough as subscription name). Note that shared subscriptions may also be durable, so this flag can (and often will) be combined with subscriptionDurab [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "task-executor" : {
+          "description" : "Allows you to specify a custom task executor for consuming messages.",
+          "type" : "string"
+        },
+        "test-connection-on-startup" : {
+          "description" : "Specifies whether to test the connection on startup. This ensures that when Camel starts that all the JMS consumers have a valid connection to the JMS broker. If a connection cannot be granted then Camel throws an exception on startup. This ensures that Camel is not started with failed connections. The JMS producers is tested as well.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-to-live" : {
+          "description" : "When sending messages, specifies the time-to-live of the message (in milliseconds).",
+          "default" : "-1",
+          "type" : "integer"
+        },
+        "transacted" : {
+          "description" : "Specifies whether to use transacted mode",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transacted-in-out" : {
+          "description" : "Specifies whether InOut operations (request reply) default to using transacted mode If this flag is set to true, then Spring JmsTemplate will have sessionTransacted set to true, and the acknowledgeMode as transacted on the JmsTemplate used for InOut operations. Note from Spring JMS: that within a JTA transaction, the parameters passed to createQueue, createTopic methods are not taken into account. Depending on the Java EE transaction context, the container make [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transaction-manager" : {
+          "description" : "The Spring transaction manager to use.",
+          "type" : "string"
+        },
+        "transaction-name" : {
+          "description" : "The name of the transaction to use.",
+          "type" : "string"
+        },
+        "transaction-timeout" : {
+          "description" : "The timeout value of the transaction (in seconds), if using transacted mode.",
+          "default" : "-1",
+          "type" : "integer"
+        },
+        "transfer-exception" : {
+          "description" : "If enabled and you are using Request Reply messaging (InOut) and an Exchange failed on the consumer side, then the caused Exception will be send back in response as a javax.jms.ObjectMessage. If the client is Camel, the returned Exception is rethrown. This allows you to use Camel JMS as a bridge in your routing - for example, using persistent queues to enable robust routing. Notice that if you also have transferExchange enabled, this option takes precedence. Th [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transfer-exchange" : {
+          "description" : "You can transfer the exchange over the wire instead of just the body and headers. The following fields are transferred: In body, Out body, Fault body, In headers, Out headers, Fault headers, exchange properties, exchange exception. This requires that the objects are serializable. Camel will exclude any non-serializable objects and log it at WARN level. You must enable this option on both the producer and consumer side, so Camel knows the payloads is an Exchange [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "use-message-id-as-correlation-id" : {
+          "description" : "Specifies whether JMSMessageID should always be used as JMSCorrelationID for InOut messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "username" : {
+          "description" : "Username to use with the ConnectionFactory. You can also configure username/password directly on the ConnectionFactory.",
+          "type" : "string"
+        },
+        "wait-for-provision-correlation-to-be-updated-counter" : {
+          "description" : "Number of times to wait for provisional correlation id to be updated to the actual correlation id when doing request/reply over JMS and when the option useMessageIDAsCorrelationID is enabled.",
+          "default" : "50",
+          "type" : "integer"
+        },
+        "wait-for-provision-correlation-to-be-updated-thread-sleeping-time" : {
+          "description" : "Interval in millis to sleep each time while waiting for provisional correlation id to be updated.",
+          "default" : "100",
+          "type" : "string"
+        }
+      }
+    },
+    "ahc" : {
+      "type" : "object",
+      "required" : [ "httpUri" ],
+      "properties" : {
+        "http-uri" : {
+          "description" : "The URI to use such as http://hostname:port/path",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "binding" : {
+          "description" : "To use a custom AhcBinding which allows to control how to bind between AHC and Camel.",
+          "type" : "string"
+        },
+        "bridge-endpoint" : {
+          "description" : "If the option is true, then the Exchange.HTTP_URI header is ignored, and use the endpoint's URI for request. You may also set the throwExceptionOnFailure to be false to let the AhcProducer send all the fault response back.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "buffer-size" : {
+          "description" : "The initial in-memory buffer size used when transferring data between Camel and AHC Client.",
+          "default" : "4096",
+          "type" : "integer"
+        },
+        "client-config" : {
+          "description" : "To configure the AsyncHttpClient to use a custom com.ning.http.client.AsyncHttpClientConfig instance.",
+          "type" : "string"
+        },
+        "client-config-options" : {
+          "description" : "To configure the AsyncHttpClientConfig using the key/values from the Map.",
+          "type" : "string"
+        },
+        "client-config-realm-options" : {
+          "description" : "To configure the AsyncHttpClientConfig Realm using the key/values from the Map.",
+          "type" : "string"
+        },
+        "connection-close" : {
+          "description" : "Define if the Connection Close header has to be added to HTTP Request. This parameter is false by default",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "cookie-handler" : {
+          "description" : "Configure a cookie handler to maintain a HTTP session",
+          "type" : "string"
+        },
+        "header-filter-strategy" : {
+          "description" : "To use a custom HeaderFilterStrategy to filter header to and from Camel message.",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "ssl-context-parameters" : {
+          "description" : "Reference to a org.apache.camel.support.jsse.SSLContextParameters in the Registry. This reference overrides any configured SSLContextParameters at the component level. See Using the JSSE Configuration Utility. Note that configuring this option will override any SSL/TLS configuration options provided through the clientConfig option at the endpoint or component level.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "throw-exception-on-failure" : {
+          "description" : "Option to disable throwing the AhcOperationFailedException in case of failed responses from the remote server. This allows you to get all responses regardless of the HTTP status code.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "transfer-exception" : {
+          "description" : "If enabled and an Exchange failed processing on the consumer side, and if the caused Exception was send back serialized in the response as a application/x-java-serialized-object content type (for example using Jetty or Servlet Camel components). On the producer side the exception will be deserialized and thrown as is, instead of the AhcOperationFailedException. The caused exception is required to be serialized. This is by default turned off. If you enable this  [...]
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "ahc-ws" : {
+      "type" : "object",
+      "required" : [ "httpUri" ],
+      "properties" : {
+        "http-uri" : {
+          "description" : "The URI to use such as http://hostname:port/path",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "binding" : {
+          "description" : "To use a custom AhcBinding which allows to control how to bind between AHC and Camel.",
+          "type" : "string"
+        },
+        "bridge-endpoint" : {
+          "description" : "If the option is true, then the Exchange.HTTP_URI header is ignored, and use the endpoint's URI for request. You may also set the throwExceptionOnFailure to be false to let the AhcProducer send all the fault response back.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "buffer-size" : {
+          "description" : "The initial in-memory buffer size used when transferring data between Camel and AHC Client.",
+          "default" : "4096",
+          "type" : "integer"
+        },
+        "client-config" : {
+          "description" : "To configure the AsyncHttpClient to use a custom com.ning.http.client.AsyncHttpClientConfig instance.",
+          "type" : "string"
+        },
+        "client-config-options" : {
+          "description" : "To configure the AsyncHttpClientConfig using the key/values from the Map.",
+          "type" : "string"
+        },
+        "client-config-realm-options" : {
+          "description" : "To configure the AsyncHttpClientConfig Realm using the key/values from the Map.",
+          "type" : "string"
+        },
+        "connection-close" : {
+          "description" : "Define if the Connection Close header has to be added to HTTP Request. This parameter is false by default",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "cookie-handler" : {
+          "description" : "Configure a cookie handler to maintain a HTTP session",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "header-filter-strategy" : {
+          "description" : "To use a custom HeaderFilterStrategy to filter header to and from Camel message.",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "send-message-on-error" : {
+          "description" : "Whether to send an message if the web-socket listener received an error.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "ssl-context-parameters" : {
+          "description" : "Reference to a org.apache.camel.support.jsse.SSLContextParameters in the Registry. This reference overrides any configured SSLContextParameters at the component level. See Using the JSSE Configuration Utility. Note that configuring this option will override any SSL/TLS configuration options provided through the clientConfig option at the endpoint or component level.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "throw-exception-on-failure" : {
+          "description" : "Option to disable throwing the AhcOperationFailedException in case of failed responses from the remote server. This allows you to get all responses regardless of the HTTP status code.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "transfer-exception" : {
+          "description" : "If enabled and an Exchange failed processing on the consumer side, and if the caused Exception was send back serialized in the response as a application/x-java-serialized-object content type (for example using Jetty or Servlet Camel components). On the producer side the exception will be deserialized and thrown as is, instead of the AhcOperationFailedException. The caused exception is required to be serialized. This is by default turned off. If you enable this  [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "use-streaming" : {
+          "description" : "To enable streaming to send data as multiple text fragments.",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "ahc-wss" : {
+      "type" : "object",
+      "$ref" : "#/definitions/ahc-ws"
+    },
+    "amqp" : {
+      "type" : "object",
+      "required" : [ "destinationName" ],
+      "properties" : {
+        "destination-name" : {
+          "description" : "Name of the queue or topic to use as destination",
+          "type" : "string"
+        },
+        "destination-type" : {
+          "description" : "The kind of destination to use",
+          "default" : "queue",
+          "enum" : [ "queue", "topic", "temp-queue", "temp-topic" ],
+          "type" : "string"
+        },
+        "accept-messages-while-stopping" : {
+          "description" : "Specifies whether the consumer accept messages while it is stopping. You may consider enabling this option, if you start and stop JMS routes at runtime, while there are still messages enqueued on the queue. If this option is false, and you stop the JMS route, then messages may be rejected, and the JMS broker would have to attempt redeliveries, which yet again may be rejected, and eventually the message may be moved at a dead letter queue on the JMS broker. To a [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "acknowledgement-mode-name" : {
+          "description" : "The JMS acknowledgement name, which is one of: SESSION_TRANSACTED, CLIENT_ACKNOWLEDGE, AUTO_ACKNOWLEDGE, DUPS_OK_ACKNOWLEDGE",
+          "default" : "AUTO_ACKNOWLEDGE",
+          "enum" : [ "SESSION_TRANSACTED", "CLIENT_ACKNOWLEDGE", "AUTO_ACKNOWLEDGE", "DUPS_OK_ACKNOWLEDGE" ],
+          "type" : "string"
+        },
+        "allow-additional-headers" : {
+          "description" : "This option is used to allow additional headers which may have values that are invalid according to JMS specification. For example some message systems such as WMQ do this with header names using prefix JMS_IBM_MQMD_ containing values with byte array or other invalid types. You can specify multiple header names separated by comma, and use as suffix for wildcard matching.",
+          "type" : "string"
+        },
+        "allow-null-body" : {
+          "description" : "Whether to allow sending messages with no body. If this option is false and the message body is null, then an JMSException is thrown.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "allow-reply-manager-quick-stop" : {
+          "description" : "Whether the DefaultMessageListenerContainer used in the reply managers for request-reply messaging allow the DefaultMessageListenerContainer.runningAllowed flag to quick stop in case JmsConfiguration#isAcceptMessagesWhileStopping is enabled, and org.apache.camel.CamelContext is currently being stopped. This quick stop ability is enabled by default in the regular JMS consumers but to enable for reply managers you must enable this flag.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "allow-serialized-headers" : {
+          "description" : "Controls whether or not to include serialized headers. Applies only when transferExchange is true. This requires that the objects are serializable. Camel will exclude any non-serializable objects and log it at WARN level.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "always-copy-message" : {
+          "description" : "If true, Camel will always make a JMS message copy of the message when it is passed to the producer for sending. Copying the message is needed in some situations, such as when a replyToDestinationSelectorName is set (incidentally, Camel will set the alwaysCopyMessage option to true, if a replyToDestinationSelectorName is set)",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "artemis-streaming-enabled" : {
+          "description" : "Whether optimizing for Apache Artemis streaming mode.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "async-consumer" : {
+          "description" : "Whether the JmsConsumer processes the Exchange asynchronously. If enabled then the JmsConsumer may pickup the next message from the JMS queue, while the previous message is being processed asynchronously (by the Asynchronous Routing Engine). This means that messages may be processed not 100% strictly in order. If disabled (as default) then the Exchange is fully processed before the JmsConsumer will pickup the next message from the JMS queue. Note if transacted  [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "async-start-listener" : {
+          "description" : "Whether to startup the JmsConsumer message listener asynchronously, when starting a route. For example if a JmsConsumer cannot get a connection to a remote JMS broker, then it may block while retrying and/or failover. This will cause Camel to block while starting routes. By setting this option to true, you will let routes startup, while the JmsConsumer connects to the JMS broker using a dedicated thread in asynchronous mode. If this option is used, then beware  [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "async-stop-listener" : {
+          "description" : "Whether to stop the JmsConsumer message listener asynchronously, when stopping a route.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "auto-startup" : {
+          "description" : "Specifies whether the consumer container should auto-startup.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "cache-level" : {
+          "description" : "Sets the cache level by ID for the underlying JMS resources. See cacheLevelName option for more details.",
+          "type" : "integer"
+        },
+        "cache-level-name" : {
+          "description" : "Sets the cache level by name for the underlying JMS resources. Possible values are: CACHE_AUTO, CACHE_CONNECTION, CACHE_CONSUMER, CACHE_NONE, and CACHE_SESSION. The default setting is CACHE_AUTO. See the Spring documentation and Transactions Cache Levels for more information.",
+          "default" : "CACHE_AUTO",
+          "enum" : [ "CACHE_AUTO", "CACHE_CONNECTION", "CACHE_CONSUMER", "CACHE_NONE", "CACHE_SESSION" ],
+          "type" : "string"
+        },
+        "client-id" : {
+          "description" : "Sets the JMS client ID to use. Note that this value, if specified, must be unique and can only be used by a single JMS connection instance. It is typically only required for durable topic subscriptions. If using Apache ActiveMQ you may prefer to use Virtual Topics instead.",
+          "type" : "string"
+        },
+        "concurrent-consumers" : {
+          "description" : "Specifies the default number of concurrent consumers when consuming from JMS (not for request/reply over JMS). See also the maxMessagesPerTask option to control dynamic scaling up/down of threads. When doing request/reply over JMS then the option replyToConcurrentConsumers is used to control number of concurrent consumers on the reply message listener.",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "connection-factory" : {
+          "description" : "The connection factory to be use. A connection factory must be configured either on the component or endpoint.",
+          "type" : "string"
+        },
+        "consumer-type" : {
+          "description" : "The consumer type to use, which can be one of: Simple, Default, or Custom. The consumer type determines which Spring JMS listener to use. Default will use org.springframework.jms.listener.DefaultMessageListenerContainer, Simple will use org.springframework.jms.listener.SimpleMessageListenerContainer. When Custom is specified, the MessageListenerContainerFactory defined by the messageListenerContainerFactory option will determine what org.springframework.jms.lis [...]
+          "default" : "Default",
+          "enum" : [ "Simple", "Default", "Custom" ],
+          "type" : "string"
+        },
+        "correlation-property" : {
+          "description" : "When using InOut exchange pattern use this JMS property instead of JMSCorrelationID JMS property to correlate messages. If set messages will be correlated solely on the value of this property JMSCorrelationID property will be ignored and not set by Camel.",
+          "type" : "string"
+        },
+        "default-task-executor-type" : {
+          "description" : "Specifies what default TaskExecutor type to use in the DefaultMessageListenerContainer, for both consumer endpoints and the ReplyTo consumer of producer endpoints. Possible values: SimpleAsync (uses Spring's SimpleAsyncTaskExecutor) or ThreadPool (uses Spring's ThreadPoolTaskExecutor with optimal values - cached threadpool-like). If not set, it defaults to the previous behaviour, which uses a cached thread pool for consumer endpoints and SimpleAsync for reply c [...]
+          "enum" : [ "ThreadPool", "SimpleAsync" ],
+          "type" : "string"
+        },
+        "delivery-delay" : {
+          "description" : "Sets delivery delay to use for send calls for JMS. This option requires JMS 2.0 compliant broker.",
+          "default" : "-1",
+          "type" : "integer"
+        },
+        "delivery-mode" : {
+          "description" : "Specifies the delivery mode to be used. Possibles values are those defined by javax.jms.DeliveryMode. NON_PERSISTENT = 1 and PERSISTENT = 2.",
+          "enum" : [ "1", "2" ],
+          "type" : "integer"
+        },
+        "delivery-persistent" : {
+          "description" : "Specifies whether persistent delivery is used by default.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "destination-resolver" : {
+          "description" : "A pluggable org.springframework.jms.support.destination.DestinationResolver that allows you to use your own resolver (for example, to lookup the real destination in a JNDI registry).",
+          "type" : "string"
+        },
+        "disable-reply-to" : {
+          "description" : "Specifies whether Camel ignores the JMSReplyTo header in messages. If true, Camel does not send a reply back to the destination specified in the JMSReplyTo header. You can use this option if you want Camel to consume from a route and you do not want Camel to automatically send back a reply message because another component in your code handles the reply message. You can also use this option if you want to use Camel as a proxy between different message brokers a [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "disable-time-to-live" : {
+          "description" : "Use this option to force disabling time to live. For example when you do request/reply over JMS, then Camel will by default use the requestTimeout value as time to live on the message being sent. The problem is that the sender and receiver systems have to have their clocks synchronized, so they are in sync. This is not always so easy to archive. So you can use disableTimeToLive=true to not set a time to live value on the sent message. Then the message will not  [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "durable-subscription-name" : {
+          "description" : "The durable subscriber name for specifying durable topic subscriptions. The clientId option must be configured as well.",
+          "type" : "string"
+        },
+        "eager-loading-of-properties" : {
+          "description" : "Enables eager loading of JMS properties and payload as soon as a message is loaded which generally is inefficient as the JMS properties may not be required but sometimes can catch early any issues with the underlying JMS provider and the use of JMS properties. See also the option eagerPoisonBody.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "eager-poison-body" : {
+          "description" : "If eagerLoadingOfProperties is enabled and the JMS message payload (JMS body or JMS properties) is poison (cannot be read/mapped), then set this text as the message body instead so the message can be processed (the cause of the poison are already stored as exception on the Exchange). This can be turned off by setting eagerPoisonBody=false. See also the option eagerLoadingOfProperties.",
+          "default" : "Poison JMS message due to ${exception.message}",
+          "type" : "string"
+        },
+        "error-handler" : {
+          "description" : "Specifies a org.springframework.util.ErrorHandler to be invoked in case of any uncaught exceptions thrown while processing a Message. By default these exceptions will be logged at the WARN level, if no errorHandler has been configured. You can configure logging level and whether stack traces should be logged using errorHandlerLoggingLevel and errorHandlerLogStackTrace options. This makes it much easier to configure, than having to code a custom errorHandler.",
+          "type" : "string"
+        },
+        "error-handler-log-stack-trace" : {
+          "description" : "Allows to control whether stacktraces should be logged or not, by the default errorHandler.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "error-handler-logging-level" : {
+          "description" : "Allows to configure the default errorHandler logging level for logging uncaught exceptions.",
+          "default" : "WARN",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exception-listener" : {
+          "description" : "Specifies the JMS Exception Listener that is to be notified of any underlying JMS exceptions.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "explicit-qos-enabled" : {
+          "description" : "Set if the deliveryMode, priority or timeToLive qualities of service should be used when sending messages. This option is based on Spring's JmsTemplate. The deliveryMode, priority and timeToLive options are applied to the current endpoint. This contrasts with the preserveMessageQos option, which operates at message granularity, reading QoS properties exclusively from the Camel In message headers.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "expose-listener-session" : {
+          "description" : "Specifies whether the listener session should be exposed when consuming messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "force-send-original-message" : {
+          "description" : "When using mapJmsMessage=false Camel will create a new JMS message to send to a new JMS destination if you touch the headers (get or set) during the route. Set this option to true to force Camel to send the original JMS message that was received.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "format-date-headers-to-iso8601" : {
+          "description" : "Sets whether JMS date properties should be formatted according to the ISO 8601 standard.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "header-filter-strategy" : {
+          "description" : "To use a custom HeaderFilterStrategy to filter header to and from Camel message.",
+          "type" : "string"
+        },
+        "idle-consumer-limit" : {
+          "description" : "Specify the limit for the number of consumers that are allowed to be idle at any given time.",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "idle-task-execution-limit" : {
+          "description" : "Specifies the limit for idle executions of a receive task, not having received any message within its execution. If this limit is reached, the task will shut down and leave receiving to other executing tasks (in the case of dynamic scheduling; see the maxConcurrentConsumers setting). There is additional doc available from Spring.",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "include-all-jmsx-properties" : {
+          "description" : "Whether to include all JMSXxxx properties when mapping from JMS to Camel Message. Setting this to true will include properties such as JMSXAppID, and JMSXUserID etc. Note: If you are using a custom headerFilterStrategy then this option does not apply.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "include-sent-jms-message-id" : {
+          "description" : "Only applicable when sending to JMS destination using InOnly (eg fire and forget). Enabling this option will enrich the Camel Exchange with the actual JMSMessageID that was used by the JMS client when the message was sent to the JMS destination.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "jms-key-format-strategy" : {
+          "description" : "Pluggable strategy for encoding and decoding JMS keys so they can be compliant with the JMS specification. Camel provides two implementations out of the box: default and passthrough. The default strategy will safely marshal dots and hyphens (. and -). The passthrough strategy leaves the key as is. Can be used for JMS brokers which do not care whether JMS header keys contain illegal characters. You can provide your own implementation of the org.apache.camel.comp [...]
+          "enum" : [ "default", "passthrough" ],
+          "type" : "string"
+        },
+        "jms-message-type" : {
+          "description" : "Allows you to force the use of a specific javax.jms.Message implementation for sending JMS messages. Possible values are: Bytes, Map, Object, Stream, Text. By default, Camel would determine which JMS message type to use from the In body type. This option allows you to specify it.",
+          "enum" : [ "Bytes", "Map", "Object", "Stream", "Text" ],
+          "type" : "string"
+        },
+        "lazy-create-transaction-manager" : {
+          "description" : "If true, Camel will create a JmsTransactionManager, if there is no transactionManager injected when option transacted=true.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "map-jms-message" : {
+          "description" : "Specifies whether Camel should auto map the received JMS message to a suited payload type, such as javax.jms.TextMessage to a String etc.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "max-concurrent-consumers" : {
+          "description" : "Specifies the maximum number of concurrent consumers when consuming from JMS (not for request/reply over JMS). See also the maxMessagesPerTask option to control dynamic scaling up/down of threads. When doing request/reply over JMS then the option replyToMaxConcurrentConsumers is used to control number of concurrent consumers on the reply message listener.",
+          "type" : "integer"
+        },
+        "max-messages-per-task" : {
+          "description" : "The number of messages per task. -1 is unlimited. If you use a range for concurrent consumers (eg min max), then this option can be used to set a value to eg 100 to control how fast the consumers will shrink when less work is required.",
+          "default" : "-1",
+          "type" : "integer"
+        },
+        "message-converter" : {
+          "description" : "To use a custom Spring org.springframework.jms.support.converter.MessageConverter so you can be in control how to map to/from a javax.jms.Message.",
+          "type" : "string"
+        },
+        "message-created-strategy" : {
+          "description" : "To use the given MessageCreatedStrategy which are invoked when Camel creates new instances of javax.jms.Message objects when Camel is sending a JMS message.",
+          "type" : "string"
+        },
+        "message-id-enabled" : {
+          "description" : "When sending, specifies whether message IDs should be added. This is just an hint to the JMS broker. If the JMS provider accepts this hint, these messages must have the message ID set to null; if the provider ignores the hint, the message ID must be set to its normal unique value.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "message-listener-container-factory" : {
+          "description" : "Registry ID of the MessageListenerContainerFactory used to determine what org.springframework.jms.listener.AbstractMessageListenerContainer to use to consume messages. Setting this will automatically set consumerType to Custom.",
+          "type" : "string"
+        },
+        "message-timestamp-enabled" : {
+          "description" : "Specifies whether timestamps should be enabled by default on sending messages. This is just an hint to the JMS broker. If the JMS provider accepts this hint, these messages must have the timestamp set to zero; if the provider ignores the hint the timestamp must be set to its normal value.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "password" : {
+          "description" : "Password to use with the ConnectionFactory. You can also configure username/password directly on the ConnectionFactory.",
+          "type" : "string"
+        },
+        "preserve-message-qos" : {
+          "description" : "Set to true, if you want to send message using the QoS settings specified on the message, instead of the QoS settings on the JMS endpoint. The following three headers are considered JMSPriority, JMSDeliveryMode, and JMSExpiration. You can provide all or only some of them. If not provided, Camel will fall back to use the values from the endpoint instead. So, when using this option, the headers override the values from the endpoint. The explicitQosEnabled option, [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "priority" : {
+          "description" : "Values greater than 1 specify the message priority when sending (where 0 is the lowest priority and 9 is the highest). The explicitQosEnabled option must also be enabled in order for this option to have any effect.",
+          "default" : "4",
+          "enum" : [ "1", "2", "3", "4", "5", "6", "7", "8", "9" ],
+          "type" : "integer"
+        },
+        "pub-sub-no-local" : {
+          "description" : "Specifies whether to inhibit the delivery of messages published by its own connection.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "receive-timeout" : {
+          "description" : "The timeout for receiving messages (in milliseconds).",
+          "default" : "1000",
+          "type" : "string"
+        },
+        "recovery-interval" : {
+          "description" : "Specifies the interval between recovery attempts, i.e. when a connection is being refreshed, in milliseconds. The default is 5000 ms, that is, 5 seconds.",
+          "default" : "5000",
+          "type" : "string"
+        },
+        "reply-to" : {
+          "description" : "Provides an explicit ReplyTo destination, which overrides any incoming value of Message.getJMSReplyTo().",
+          "type" : "string"
+        },
+        "reply-to-cache-level-name" : {
+          "description" : "Sets the cache level by name for the reply consumer when doing request/reply over JMS. This option only applies when using fixed reply queues (not temporary). Camel will by default use: CACHE_CONSUMER for exclusive or shared w/ replyToSelectorName. And CACHE_SESSION for shared without replyToSelectorName. Some JMS brokers such as IBM WebSphere may require to set the replyToCacheLevelName=CACHE_NONE to work. Note: If using temporary queues then CACHE_NONE is not [...]
+          "enum" : [ "CACHE_AUTO", "CACHE_CONNECTION", "CACHE_CONSUMER", "CACHE_NONE", "CACHE_SESSION" ],
+          "type" : "string"
+        },
+        "reply-to-concurrent-consumers" : {
+          "description" : "Specifies the default number of concurrent consumers when doing request/reply over JMS. See also the maxMessagesPerTask option to control dynamic scaling up/down of threads.",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "reply-to-delivery-persistent" : {
+          "description" : "Specifies whether to use persistent delivery by default for replies.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "reply-to-destination-selector-name" : {
+          "description" : "Sets the JMS Selector using the fixed name to be used so you can filter out your own replies from the others when using a shared queue (that is, if you are not using a temporary reply queue).",
+          "type" : "string"
+        },
+        "reply-to-max-concurrent-consumers" : {
+          "description" : "Specifies the maximum number of concurrent consumers when using request/reply over JMS. See also the maxMessagesPerTask option to control dynamic scaling up/down of threads.",
+          "type" : "integer"
+        },
+        "reply-to-on-timeout-max-concurrent-consumers" : {
+          "description" : "Specifies the maximum number of concurrent consumers for continue routing when timeout occurred when using request/reply over JMS.",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "reply-to-override" : {
+          "description" : "Provides an explicit ReplyTo destination in the JMS message, which overrides the setting of replyTo. It is useful if you want to forward the message to a remote Queue and receive the reply message from the ReplyTo destination.",
+          "type" : "string"
+        },
+        "reply-to-same-destination-allowed" : {
+          "description" : "Whether a JMS consumer is allowed to send a reply message to the same destination that the consumer is using to consume from. This prevents an endless loop by consuming and sending back the same message to itself.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "reply-to-type" : {
+          "description" : "Allows for explicitly specifying which kind of strategy to use for replyTo queues when doing request/reply over JMS. Possible values are: Temporary, Shared, or Exclusive. By default Camel will use temporary queues. However if replyTo has been configured, then Shared is used by default. This option allows you to use exclusive queues instead of shared ones. See Camel JMS documentation for more details, and especially the notes about the implications if running in [...]
+          "enum" : [ "Temporary", "Shared", "Exclusive" ],
+          "type" : "string"
+        },
+        "request-timeout" : {
+          "description" : "The timeout for waiting for a reply when using the InOut Exchange Pattern (in milliseconds). The default is 20 seconds. You can include the header CamelJmsRequestTimeout to override this endpoint configured timeout value, and thus have per message individual timeout values. See also the requestTimeoutCheckerInterval option.",
+          "default" : "20000",
+          "type" : "string"
+        },
+        "request-timeout-checker-interval" : {
+          "description" : "Configures how often Camel should check for timed out Exchanges when doing request/reply over JMS. By default Camel checks once per second. But if you must react faster when a timeout occurs, then you can lower this interval, to check more frequently. The timeout is determined by the option requestTimeout.",
+          "default" : "1000",
+          "type" : "string"
+        },
+        "selector" : {
+          "description" : "Sets the JMS selector to use",
+          "type" : "string"
+        },
+        "stream-message-type-enabled" : {
+          "description" : "Sets whether StreamMessage type is enabled or not. Message payloads of streaming kind such as files, InputStream, etc will either by sent as BytesMessage or StreamMessage. This option controls which kind will be used. By default BytesMessage is used which enforces the entire message payload to be read into memory. By enabling this option the message payload is read into memory in chunks and each chunk is then written to the StreamMessage until no more data.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "subscription-durable" : {
+          "description" : "Set whether to make the subscription durable. The durable subscription name to be used can be specified through the subscriptionName property. Default is false. Set this to true to register a durable subscription, typically in combination with a subscriptionName value (unless your message listener class name is good enough as subscription name). Only makes sense when listening to a topic (pub-sub domain), therefore this method switches the pubSubDomain flag as well.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "subscription-name" : {
+          "description" : "Set the name of a subscription to create. To be applied in case of a topic (pub-sub domain) with a shared or durable subscription. The subscription name needs to be unique within this client's JMS client id. Default is the class name of the specified message listener. Note: Only 1 concurrent consumer (which is the default of this message listener container) is allowed for each subscription, except for a shared subscription (which requires JMS 2.0).",
+          "type" : "string"
+        },
+        "subscription-shared" : {
+          "description" : "Set whether to make the subscription shared. The shared subscription name to be used can be specified through the subscriptionName property. Default is false. Set this to true to register a shared subscription, typically in combination with a subscriptionName value (unless your message listener class name is good enough as subscription name). Note that shared subscriptions may also be durable, so this flag can (and often will) be combined with subscriptionDurab [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "task-executor" : {
+          "description" : "Allows you to specify a custom task executor for consuming messages.",
+          "type" : "string"
+        },
+        "test-connection-on-startup" : {
+          "description" : "Specifies whether to test the connection on startup. This ensures that when Camel starts that all the JMS consumers have a valid connection to the JMS broker. If a connection cannot be granted then Camel throws an exception on startup. This ensures that Camel is not started with failed connections. The JMS producers is tested as well.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-to-live" : {
+          "description" : "When sending messages, specifies the time-to-live of the message (in milliseconds).",
+          "default" : "-1",
+          "type" : "integer"
+        },
+        "transacted" : {
+          "description" : "Specifies whether to use transacted mode",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transacted-in-out" : {
+          "description" : "Specifies whether InOut operations (request reply) default to using transacted mode If this flag is set to true, then Spring JmsTemplate will have sessionTransacted set to true, and the acknowledgeMode as transacted on the JmsTemplate used for InOut operations. Note from Spring JMS: that within a JTA transaction, the parameters passed to createQueue, createTopic methods are not taken into account. Depending on the Java EE transaction context, the container make [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transaction-manager" : {
+          "description" : "The Spring transaction manager to use.",
+          "type" : "string"
+        },
+        "transaction-name" : {
+          "description" : "The name of the transaction to use.",
+          "type" : "string"
+        },
+        "transaction-timeout" : {
+          "description" : "The timeout value of the transaction (in seconds), if using transacted mode.",
+          "default" : "-1",
+          "type" : "integer"
+        },
+        "transfer-exception" : {
+          "description" : "If enabled and you are using Request Reply messaging (InOut) and an Exchange failed on the consumer side, then the caused Exception will be send back in response as a javax.jms.ObjectMessage. If the client is Camel, the returned Exception is rethrown. This allows you to use Camel JMS as a bridge in your routing - for example, using persistent queues to enable robust routing. Notice that if you also have transferExchange enabled, this option takes precedence. Th [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transfer-exchange" : {
+          "description" : "You can transfer the exchange over the wire instead of just the body and headers. The following fields are transferred: In body, Out body, Fault body, In headers, Out headers, Fault headers, exchange properties, exchange exception. This requires that the objects are serializable. Camel will exclude any non-serializable objects and log it at WARN level. You must enable this option on both the producer and consumer side, so Camel knows the payloads is an Exchange [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "use-message-id-as-correlation-id" : {
+          "description" : "Specifies whether JMSMessageID should always be used as JMSCorrelationID for InOut messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "username" : {
+          "description" : "Username to use with the ConnectionFactory. You can also configure username/password directly on the ConnectionFactory.",
+          "type" : "string"
+        },
+        "wait-for-provision-correlation-to-be-updated-counter" : {
+          "description" : "Number of times to wait for provisional correlation id to be updated to the actual correlation id when doing request/reply over JMS and when the option useMessageIDAsCorrelationID is enabled.",
+          "default" : "50",
+          "type" : "integer"
+        },
+        "wait-for-provision-correlation-to-be-updated-thread-sleeping-time" : {
+          "description" : "Interval in millis to sleep each time while waiting for provisional correlation id to be updated.",
+          "default" : "100",
+          "type" : "string"
+        }
+      }
+    },
+    "apns" : {
+      "type" : "object",
+      "properties" : {
+        "name" : {
+          "description" : "Name of the endpoint",
+          "type" : "string"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "tokens" : {
+          "description" : "Configure this property in case you want to statically declare tokens related to devices you want to notify. Tokens are separated by comma.",
+          "type" : "string"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        }
+      }
+    },
+    "arangodb" : {
+      "type" : "object",
+      "required" : [ "database" ],
+      "properties" : {
+        "database" : {
+          "description" : "database name",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "document-collection" : {
+          "description" : "Collection name, when using ArangoDb as a Document Database. Set the documentCollection name when using the CRUD operation on the document database collections (SAVE_DOCUMENT , FIND_DOCUMENT_BY_KEY, UPDATE_DOCUMENT, DELETE_DOCUMENT).",
+          "type" : "string"
+        },
+        "edge-collection" : {
+          "description" : "Collection name of vertices, when using ArangoDb as a Graph Database. Set the edgeCollection name to perform CRUD operation on edges using these operations : SAVE_VERTEX, FIND_VERTEX_BY_KEY, UPDATE_VERTEX, DELETE_VERTEX. The graph attribute is mandatory.",
+          "type" : "string"
+        },
+        "graph" : {
+          "description" : "Graph name, when using ArangoDb as a Graph Database. Combine this attribute with one of the two attributes vertexCollection and edgeCollection.",
+          "type" : "string"
+        },
+        "host" : {
+          "description" : "ArangoDB host. If host and port are default, this field is Optional.",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "Operations to perform on ArangoDb. For the operation AQL_QUERY, no need to specify a collection or graph.",
+          "enum" : [ "SAVE_DOCUMENT", "FIND_DOCUMENT_BY_KEY", "UPDATE_DOCUMENT", "DELETE_DOCUMENT", "AQL_QUERY", "SAVE_VERTEX", "FIND_VERTEX_BY_KEY", "UPDATE_VERTEX", "DELETE_VERTEX", "SAVE_EDGE", "FIND_EDGE_BY_KEY", "UPDATE_EDGE", "DELETE_EDGE" ],
+          "type" : "string"
+        },
+        "password" : {
+          "description" : "ArangoDB password. If user and password are default, this field is Optional.",
+          "type" : "string"
+        },
+        "port" : {
+          "description" : "ArangoDB exposed port. If host and port are default, this field is Optional.",
+          "type" : "integer"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "user" : {
+          "description" : "ArangoDB user. If user and password are default, this field is Optional.",
+          "type" : "string"
+        },
+        "vertex-collection" : {
+          "description" : "Collection name of vertices, when using ArangoDb as a Graph Database. Set the vertexCollection name to perform CRUD operation on vertices using these operations : SAVE_EDGE, FIND_EDGE_BY_KEY, UPDATE_EDGE, DELETE_EDGE. The graph attribute is mandatory.",
+          "type" : "string"
+        }
+      }
+    },
+    "as2" : {
+      "type" : "object",
+      "required" : [ "apiName", "methodName" ],
+      "properties" : {
+        "api-name" : {
+          "description" : "What kind of operation to perform",
+          "enum" : [ "CLIENT", "SERVER" ],
+          "type" : "string"
+        },
+        "method-name" : {
+          "description" : "What sub operation to use for the selected operation",
+          "type" : "string"
+        },
+        "as2-from" : {
+          "description" : "The value of the AS2From header of AS2 message.",
+          "type" : "string"
+        },
+        "as2-message-structure" : {
+          "description" : "The structure of AS2 Message. One of: PLAIN - No encryption, no signature, SIGNED - No encryption, signature, ENCRYPTED - Encryption, no signature, ENCRYPTED_SIGNED - Encryption, signature",
+          "enum" : [ "PLAIN", "SIGNED", "ENCRYPTED", "SIGNED_ENCRYPTED", "PLAIN_COMPRESSED", "SIGNED_COMPRESSED", "ENCRYPTED_COMPRESSED", "ENCRYPTED_COMPRESSED_SIGNED" ],
+          "type" : "string"
+        },
+        "as2-to" : {
+          "description" : "The value of the AS2To header of AS2 message.",
+          "type" : "string"
+        },
+        "as2-version" : {
+          "description" : "The version of the AS2 protocol.",
+          "default" : "1.1",
+          "enum" : [ "1.0", "1.1" ],
+          "type" : "string"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "client-fqdn" : {
+          "description" : "The Client Fully Qualified Domain Name (FQDN). Used in message ids sent by endpoint.",
+          "default" : "camel.apache.org",
+          "type" : "string"
+        },
+        "compression-algorithm" : {
+          "description" : "The algorithm used to compress EDI message.",
+          "enum" : [ "ZLIB" ],
+          "type" : "string"
+        },
+        "decrypting-private-key" : {
+          "description" : "The key used to encrypt the EDI message.",
+          "type" : "string"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "disposition-notification-to" : {
+          "description" : "The value of the Disposition-Notification-To header. Assigning a value to this parameter requests a message disposition notification (MDN) for the AS2 message.",
+          "type" : "string"
+        },
+        "edi-message-transfer-encoding" : {
+          "description" : "The transfer encoding of EDI message.",
+          "type" : "string"
+        },
+        "edi-message-type" : {
+          "description" : "The content type of EDI message. One of application/edifact, application/edi-x12, application/edi-consent",
+          "type" : "string"
+        },
+        "encrypting-algorithm" : {
+          "description" : "The algorithm used to encrypt EDI message.",
+          "enum" : [ "AES128_CBC", "AES192_CBC", "AES256_CBC", "AES128_CCM", "AES192_CCM", "AES256_CCM", "AES128_GCM", "AES192_GCM", "AES256_GCM", "CAMELLIA128_CBC", "CAMELLIA192_CBC", "CAMELLIA256_CBC", "CAST5_CBC", "DES_CBC", "DES_EDE3_CBC", "GOST28147_GCFB", "IDEA_CBC", "RC2_CBC", "RC4", "SEED_CBC" ],
+          "type" : "string"
+        },
+        "encrypting-certificate-chain" : {
+          "description" : "The chain of certificates used to encrypt EDI message.",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "from" : {
+          "description" : "The value of the From header of AS2 message.",
+          "type" : "string"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "in-body" : {
+          "description" : "Sets the name of a parameter to be passed in the exchange In Body",
+          "type" : "string"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "mdn-message-template" : {
+          "description" : "The template used to format MDN message",
+          "type" : "string"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "request-uri" : {
+          "description" : "The request URI of EDI message.",
+          "default" : "/",
+          "type" : "string"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "server" : {
+          "description" : "The value included in the Server message header identifying the AS2 Server.",
+          "default" : "Camel AS2 Server Endpoint",
+          "type" : "string"
+        },
+        "server-fqdn" : {
+          "description" : "The Server Fully Qualified Domain Name (FQDN). Used in message ids sent by endpoint.",
+          "default" : "camel.apache.org",
+          "type" : "string"
+        },
+        "server-port-number" : {
+          "description" : "The port number of server.",
+          "type" : "integer"
+        },
+        "signed-receipt-mic-algorithms" : {
+          "description" : "The list of algorithms, in order of preference, requested to generate a message integrity check (MIC) returned in message dispostion notification (MDN)",
+          "type" : "string"
+        },
+        "signing-algorithm" : {
+          "description" : "The algorithm used to sign EDI message.",
+          "enum" : [ "SHA3_224WITHRSA", "SHA3_256WITHRSA", "SHA3_384withRSA", "SHA3_512WITHRSA", "MD5WITHRSA", "SHA1WITHRSA", "MD2WITHRSA", "SHA224WITHRSA", "SHA256WITHRSA", "SHA384WITHRSA", "SHA512WITHRSA", "RIPEMD128WITHRSA", "RIPEMD160WITHRSA", "RIPEMD256WITHRSA", "SHA224WITHDSA", "SHA256WITHDSA", "SHA384WITHDSA", "SHA512WITHDSA", "SHA3_224WITHDSA", "SHA3_256WITHDSA", "SHA3_384WITHDSA", "SHA3_512WITHDSA", "SHA1WITHDSA", "SHA3_224WITHECDSA", "SHA3_256WITHECDSA", "SHA3_384WITHECDSA", "S [...]
+          "type" : "string"
+        },
+        "signing-certificate-chain" : {
+          "description" : "The chain of certificates used to sign EDI message.",
+          "type" : "string"
+        },
+        "signing-private-key" : {
+          "description" : "The key used to sign the EDI message.",
+          "type" : "string"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "subject" : {
+          "description" : "The value of Subject header of AS2 message.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "target-hostname" : {
+          "description" : "The host name (IP or DNS name) of target host.",
+          "type" : "string"
+        },
+        "target-port-number" : {
+          "description" : "The port number of target host. -1 indicates the scheme default port.",
+          "type" : "integer"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "user-agent" : {
+          "description" : "The value included in the User-Agent message header identifying the AS2 user agent.",
+          "default" : "Camel AS2 Client Endpoint",
+          "type" : "string"
+        }
+      }
+    },
+    "asterisk" : {
+      "type" : "object",
+      "required" : [ "name", "hostname", "password", "username" ],
+      "properties" : {
+        "name" : {
+          "description" : "Name of component",
+          "type" : "string"
+        },
+        "action" : {
+          "description" : "What action to perform such as getting queue status, sip peers or extension state.",
+          "enum" : [ "QUEUE_STATUS", "SIP_PEERS", "EXTENSION_STATE" ],
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "hostname" : {
+          "description" : "The hostname of the asterisk server",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "password" : {
+          "description" : "Login password",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "username" : {
+          "description" : "Login username",
+          "type" : "string"
+        }
+      }
+    },
+    "atmos" : {
+      "type" : "object",
+      "properties" : {
+        "name" : {
+          "description" : "Atmos name",
+          "type" : "string"
+        },
+        "operation" : {
+          "description" : "Operation to perform",
+          "enum" : [ "put", "del", "search", "get", "move" ],
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "full-token-id" : {
+          "description" : "Atmos client fullTokenId",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "local-path" : {
+          "description" : "Local path to put files",
+          "type" : "string"
+        },
+        "new-remote-path" : {
+          "description" : "New path on Atmos when moving files",
+          "type" : "string"
+        },
+        "query" : {
+          "description" : "Search query on Atmos",
+          "type" : "string"
+        },
+        "remote-path" : {
+          "description" : "Where to put files on Atmos",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "The secret key to pass to the Atmos client (should be base64 encoded)",
+          "type" : "string"
+        },
+        "ssl-validation" : {
+          "description" : "Atmos SSL validation",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "uri" : {
+          "description" : "Atomos server uri",
+          "type" : "string"
+        }
+      },
+      "required" : [ "operation" ]
+    },
+    "atmosphere-websocket" : {
+      "type" : "object",
+      "required" : [ "servicePath" ],
+      "properties" : {
+        "service-path" : {
+          "description" : "Name of websocket endpoint",
+          "type" : "string"
+        },
+        "async" : {
+          "description" : "Configure the consumer to work in async mode",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "attachment-multipart-binding" : {
+          "description" : "Whether to automatic bind multipart/form-data as attachments on the Camel Exchange. The options attachmentMultipartBinding=true and disableStreamCache=false cannot work together. Remove disableStreamCache to use AttachmentMultipartBinding. This is turn off by default as this may require servlet specific configuration to enable this when using Servlet's.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-endpoint" : {
+          "description" : "If the option is true, HttpProducer will ignore the Exchange.HTTP_URI header, and use the endpoint's URI for request. You may also set the option throwExceptionOnFailure to be false to let the HttpProducer send all the fault response back.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "chunked" : {
+          "description" : "If this option is false the Servlet will disable the HTTP streaming and set the content-length header on the response",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "disable-stream-cache" : {
+          "description" : "Determines whether or not the raw input stream from Servlet is cached or not (Camel will read the stream into a in memory/overflow to file, Stream caching) cache. By default Camel will cache the Servlet input stream to support reading it multiple times to ensure it Camel can retrieve all data from the stream. However you can set this option to true when you for example need to access the raw stream, such as streaming it directly to a file or other persistent st [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "eager-check-content-available" : {
+          "description" : "Whether to eager check whether the HTTP requests has content if the content-length header is 0 or not present. This can be turned on in case HTTP clients do not send streamed data.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "file-name-ext-whitelist" : {
+          "description" : "Whitelist of accepted filename extensions for accepting uploaded files. Multiple extensions can be separated by comma, such as txt,xml.",
+          "type" : "string"
+        },
+        "header-filter-strategy" : {
+          "description" : "To use a custom HeaderFilterStrategy to filter header to and from Camel message.",
+          "type" : "string"
+        },
+        "http-binding" : {
+          "description" : "To use a custom HttpBinding to control the mapping between Camel message and HttpClient.",
+          "type" : "string"
+        },
+        "http-method-restrict" : {
+          "description" : "Used to only allow consuming if the HttpMethod matches, such as GET/POST/PUT etc. Multiple methods can be specified separated by comma.",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "map-http-message-body" : {
+          "description" : "If this option is true then IN exchange Body of the exchange will be mapped to HTTP body. Setting this to false will avoid the HTTP mapping.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "map-http-message-form-url-encoded-body" : {
+          "description" : "If this option is true then IN exchange Form Encoded body of the exchange will be mapped to HTTP. Setting this to false will avoid the HTTP Form Encoded body mapping.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "map-http-message-headers" : {
+          "description" : "If this option is true then IN exchange Headers of the exchange will be mapped to HTTP headers. Setting this to false will avoid the HTTP Headers mapping.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "match-on-uri-prefix" : {
+          "description" : "Whether or not the consumer should try to find a target consumer by matching the URI prefix if no exact match is found.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "mute-exception" : {
+          "description" : "If enabled and an Exchange failed processing on the consumer side the response's body won't contain the exception's stack trace.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "options-enabled" : {
+          "description" : "Specifies whether to enable HTTP OPTIONS for this Servlet consumer. By default OPTIONS is turned off.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "response-buffer-size" : {
+          "description" : "To use a custom buffer size on the javax.servlet.ServletResponse.",
+          "type" : "integer"
+        },
+        "send-to-all" : {
+          "description" : "Whether to send to all (broadcast) or send to a single receiver.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "servlet-name" : {
+          "description" : "Name of the servlet to use",
+          "default" : "CamelServlet",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trace-enabled" : {
+          "description" : "Specifies whether to enable HTTP TRACE for this Servlet consumer. By default TRACE is turned off.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transfer-exception" : {
+          "description" : "If enabled and an Exchange failed processing on the consumer side, and if the caused Exception was send back serialized in the response as a application/x-java-serialized-object content type. On the producer side the exception will be deserialized and thrown as is, instead of the HttpOperationFailedException. The caused exception is required to be serialized. This is by default turned off. If you enable this then be aware that Java will deserialize the incoming [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "use-streaming" : {
+          "description" : "To enable streaming to send data as multiple text fragments.",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "atom" : {
+      "type" : "object",
+      "required" : [ "feedUri" ],
+      "properties" : {
+        "feed-uri" : {
+          "description" : "The URI to the feed to poll.",
+          "type" : "string"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "feed-header" : {
+          "description" : "Sets whether to add the feed object as a header.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "filter" : {
+          "description" : "Sets whether to use filtering or not of the entries.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "last-update" : {
+          "description" : "Sets the timestamp to be used for filtering entries from the atom feeds. This options is only in conjunction with the splitEntries.",
+          "type" : "string"
+        },
+        "password" : {
+          "description" : "Sets the password to be used for basic authentication when polling from a HTTP feed.",
+          "type" : "string"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "sort-entries" : {
+          "description" : "Sets whether to sort entries by published date. Only works when splitEntries = true.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "split-entries" : {
+          "description" : "Sets whether or not entries should be sent individually or whether the entire feed should be sent as a single message",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "throttle-entries" : {
+          "description" : "Sets whether all entries identified in a single feed poll should be delivered immediately. If true, only one entry is processed per delay. Only applicable when splitEntries = true.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "username" : {
+          "description" : "Sets the username to be used for basic authentication when polling from a HTTP feed.",
+          "type" : "string"
+        }
+      }
+    },
+    "atomix-map" : {
+      "type" : "object",
+      "required" : [ "resourceName" ],
+      "properties" : {
+        "resource-name" : {
+          "description" : "The distributed resource name",
+          "type" : "string"
+        },
+        "atomix" : {
+          "description" : "The Atomix instance to use",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "configuration-uri" : {
+          "description" : "The Atomix configuration uri.",
+          "type" : "string"
+        },
+        "default-action" : {
+          "description" : "The default action.",
+          "default" : "PUT",
+          "enum" : [ "PUT", "PUT_IF_ABSENT", "GET", "CLEAR", "SIZE", "CONTAINS_KEY", "CONTAINS_VALUE", "IS_EMPTY", "ENTRY_SET", "REMOVE", "REPLACE", "VALUES" ],
+          "type" : "string"
+        },
+        "default-resource-config" : {
+          "description" : "The cluster wide default resource configuration.",
+          "type" : "string"
+        },
+        "default-resource-options" : {
+          "description" : "The local default resource options.",
+          "type" : "string"
+        },
+        "ephemeral" : {
+          "description" : "Sets if the local member should join groups as PersistentMember or not. If set to ephemeral the local member will receive an auto generated ID thus the local one is ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "key" : {
+          "description" : "The key to use if none is set in the header or to listen for events for a specific key.",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "nodes" : {
+          "description" : "The address of the nodes composing the cluster.",
+          "type" : "string"
+        },
+        "read-consistency" : {
+          "description" : "The read consistency level.",
+          "enum" : [ "ATOMIC", "ATOMIC_LEASE", "SEQUENTIAL", "LOCAL" ],
+          "type" : "string"
+        },
+        "resource-configs" : {
+          "description" : "Cluster wide resources configuration.",
+          "type" : "string"
+        },
+        "resource-options" : {
+          "description" : "Local resources configurations",
+          "type" : "string"
+        },
+        "result-header" : {
+          "description" : "The header that wil carry the result.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transport-class-name" : {
+          "description" : "The class name (fqn) of the Atomix transport",
+          "default" : "io.atomix.catalyst.transport.netty.NettyTransport",
+          "type" : "string"
+        },
+        "ttl" : {
+          "description" : "The resource ttl.",
+          "type" : "string"
+        }
+      }
+    },
+    "atomix-messaging" : {
+      "type" : "object",
+      "required" : [ "resourceName" ],
+      "properties" : {
+        "resource-name" : {
+          "description" : "The distributed resource name",
+          "type" : "string"
+        },
+        "atomix" : {
+          "description" : "The Atomix instance to use",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "broadcast-type" : {
+          "description" : "The broadcast type.",
+          "default" : "ALL",
+          "enum" : [ "ALL", "RANDOM" ],
+          "type" : "string"
+        },
+        "channel-name" : {
+          "description" : "The messaging channel name",
+          "type" : "string"
+        },
+        "configuration-uri" : {
+          "description" : "The Atomix configuration uri.",
+          "type" : "string"
+        },
+        "default-action" : {
+          "description" : "The default action.",
+          "default" : "DIRECT",
+          "enum" : [ "DIRECT", "BROADCAST" ],
+          "type" : "string"
+        },
+        "default-resource-config" : {
+          "description" : "The cluster wide default resource configuration.",
+          "type" : "string"
+        },
+        "default-resource-options" : {
+          "description" : "The local default resource options.",
+          "type" : "string"
+        },
+        "ephemeral" : {
+          "description" : "Sets if the local member should join groups as PersistentMember or not. If set to ephemeral the local member will receive an auto generated ID thus the local one is ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "member-name" : {
+          "description" : "The Atomix Group member name",
+          "type" : "string"
+        },
+        "nodes" : {
+          "description" : "The address of the nodes composing the cluster.",
+          "type" : "string"
+        },
+        "read-consistency" : {
+          "description" : "The read consistency level.",
+          "enum" : [ "ATOMIC", "ATOMIC_LEASE", "SEQUENTIAL", "LOCAL" ],
+          "type" : "string"
+        },
+        "resource-configs" : {
+          "description" : "Cluster wide resources configuration.",
+          "type" : "string"
+        },
+        "resource-options" : {
+          "description" : "Local resources configurations",
+          "type" : "string"
+        },
+        "result-header" : {
+          "description" : "The header that wil carry the result.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transport-class-name" : {
+          "description" : "The class name (fqn) of the Atomix transport",
+          "default" : "io.atomix.catalyst.transport.netty.NettyTransport",
+          "type" : "string"
+        }
+      }
+    },
+    "atomix-multimap" : {
+      "type" : "object",
+      "required" : [ "resourceName" ],
+      "properties" : {
+        "resource-name" : {
+          "description" : "The distributed resource name",
+          "type" : "string"
+        },
+        "atomix" : {
+          "description" : "The Atomix instance to use",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "configuration-uri" : {
+          "description" : "The Atomix configuration uri.",
+          "type" : "string"
+        },
+        "default-action" : {
+          "description" : "The default action.",
+          "default" : "PUT",
+          "enum" : [ "PUT", "GET", "CLEAR", "SIZE", "CONTAINS_KEY", "IS_EMPTY", "REMOVE", "REMOVE_VALUE" ],
+          "type" : "string"
+        },
+        "default-resource-config" : {
+          "description" : "The cluster wide default resource configuration.",
+          "type" : "string"
+        },
+        "default-resource-options" : {
+          "description" : "The local default resource options.",
+          "type" : "string"
+        },
+        "ephemeral" : {
+          "description" : "Sets if the local member should join groups as PersistentMember or not. If set to ephemeral the local member will receive an auto generated ID thus the local one is ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "key" : {
+          "description" : "The key to use if none is set in the header or to listen for events for a specific key.",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "nodes" : {
+          "description" : "The address of the nodes composing the cluster.",
+          "type" : "string"
+        },
+        "read-consistency" : {
+          "description" : "The read consistency level.",
+          "enum" : [ "ATOMIC", "ATOMIC_LEASE", "SEQUENTIAL", "LOCAL" ],
+          "type" : "string"
+        },
+        "resource-configs" : {
+          "description" : "Cluster wide resources configuration.",
+          "type" : "string"
+        },
+        "resource-options" : {
+          "description" : "Local resources configurations",
+          "type" : "string"
+        },
+        "result-header" : {
+          "description" : "The header that wil carry the result.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transport-class-name" : {
+          "description" : "The class name (fqn) of the Atomix transport",
+          "default" : "io.atomix.catalyst.transport.netty.NettyTransport",
+          "type" : "string"
+        },
+        "ttl" : {
+          "description" : "The resource ttl.",
+          "type" : "string"
+        }
+      }
+    },
+    "atomix-queue" : {
+      "type" : "object",
+      "required" : [ "resourceName" ],
+      "properties" : {
+        "resource-name" : {
+          "description" : "The distributed resource name",
+          "type" : "string"
+        },
+        "atomix" : {
+          "description" : "The Atomix instance to use",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "configuration-uri" : {
+          "description" : "The Atomix configuration uri.",
+          "type" : "string"
+        },
+        "default-action" : {
+          "description" : "The default action.",
+          "default" : "ADD",
+          "enum" : [ "ADD", "OFFER", "PEEK", "POLL", "CLEAR", "CONTAINS", "IS_EMPTY", "REMOVE", "SIZE" ],
+          "type" : "string"
+        },
+        "default-resource-config" : {
+          "description" : "The cluster wide default resource configuration.",
+          "type" : "string"
+        },
+        "default-resource-options" : {
+          "description" : "The local default resource options.",
+          "type" : "string"
+        },
+        "ephemeral" : {
+          "description" : "Sets if the local member should join groups as PersistentMember or not. If set to ephemeral the local member will receive an auto generated ID thus the local one is ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "nodes" : {
+          "description" : "The address of the nodes composing the cluster.",
+          "type" : "string"
+        },
+        "read-consistency" : {
+          "description" : "The read consistency level.",
+          "enum" : [ "ATOMIC", "ATOMIC_LEASE", "SEQUENTIAL", "LOCAL" ],
+          "type" : "string"
+        },
+        "resource-configs" : {
+          "description" : "Cluster wide resources configuration.",
+          "type" : "string"
+        },
+        "resource-options" : {
+          "description" : "Local resources configurations",
+          "type" : "string"
+        },
+        "result-header" : {
+          "description" : "The header that wil carry the result.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transport-class-name" : {
+          "description" : "The class name (fqn) of the Atomix transport",
+          "default" : "io.atomix.catalyst.transport.netty.NettyTransport",
+          "type" : "string"
+        }
+      }
+    },
+    "atomix-set" : {
+      "type" : "object",
+      "required" : [ "resourceName" ],
+      "properties" : {
+        "resource-name" : {
+          "description" : "The distributed resource name",
+          "type" : "string"
+        },
+        "atomix" : {
+          "description" : "The Atomix instance to use",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "configuration-uri" : {
+          "description" : "The Atomix configuration uri.",
+          "type" : "string"
+        },
+        "default-action" : {
+          "description" : "The default action.",
+          "default" : "ADD",
+          "enum" : [ "ADD", "CLEAR", "CONTAINS", "IS_EMPTY", "REMOVE", "SIZE" ],
+          "type" : "string"
+        },
+        "default-resource-config" : {
+          "description" : "The cluster wide default resource configuration.",
+          "type" : "string"
+        },
+        "default-resource-options" : {
+          "description" : "The local default resource options.",
+          "type" : "string"
+        },
+        "ephemeral" : {
+          "description" : "Sets if the local member should join groups as PersistentMember or not. If set to ephemeral the local member will receive an auto generated ID thus the local one is ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "nodes" : {
+          "description" : "The address of the nodes composing the cluster.",
+          "type" : "string"
+        },
+        "read-consistency" : {
+          "description" : "The read consistency level.",
+          "enum" : [ "ATOMIC", "ATOMIC_LEASE", "SEQUENTIAL", "LOCAL" ],
+          "type" : "string"
+        },
+        "resource-configs" : {
+          "description" : "Cluster wide resources configuration.",
+          "type" : "string"
+        },
+        "resource-options" : {
+          "description" : "Local resources configurations",
+          "type" : "string"
+        },
+        "result-header" : {
+          "description" : "The header that wil carry the result.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transport-class-name" : {
+          "description" : "The class name (fqn) of the Atomix transport",
+          "default" : "io.atomix.catalyst.transport.netty.NettyTransport",
+          "type" : "string"
+        },
+        "ttl" : {
+          "description" : "The resource ttl.",
+          "type" : "string"
+        }
+      }
+    },
+    "atomix-value" : {
+      "type" : "object",
+      "required" : [ "resourceName" ],
+      "properties" : {
+        "resource-name" : {
+          "description" : "The distributed resource name",
+          "type" : "string"
+        },
+        "atomix" : {
+          "description" : "The Atomix instance to use",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "configuration-uri" : {
+          "description" : "The Atomix configuration uri.",
+          "type" : "string"
+        },
+        "default-action" : {
+          "description" : "The default action.",
+          "default" : "SET",
+          "enum" : [ "SET", "GET", "GET_AND_SET", "COMPARE_AND_SET" ],
+          "type" : "string"
+        },
+        "default-resource-config" : {
+          "description" : "The cluster wide default resource configuration.",
+          "type" : "string"
+        },
+        "default-resource-options" : {
+          "description" : "The local default resource options.",
+          "type" : "string"
+        },
+        "ephemeral" : {
+          "description" : "Sets if the local member should join groups as PersistentMember or not. If set to ephemeral the local member will receive an auto generated ID thus the local one is ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "nodes" : {
+          "description" : "The address of the nodes composing the cluster.",
+          "type" : "string"
+        },
+        "read-consistency" : {
+          "description" : "The read consistency level.",
+          "enum" : [ "ATOMIC", "ATOMIC_LEASE", "SEQUENTIAL", "LOCAL" ],
+          "type" : "string"
+        },
+        "resource-configs" : {
+          "description" : "Cluster wide resources configuration.",
+          "type" : "string"
+        },
+        "resource-options" : {
+          "description" : "Local resources configurations",
+          "type" : "string"
+        },
+        "result-header" : {
+          "description" : "The header that wil carry the result.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "transport-class-name" : {
+          "description" : "The class name (fqn) of the Atomix transport",
+          "default" : "io.atomix.catalyst.transport.netty.NettyTransport",
+          "type" : "string"
+        },
+        "ttl" : {
+          "description" : "The resource ttl.",
+          "type" : "string"
+        }
+      }
+    },
+    "avro" : {
+      "type" : "object",
+      "required" : [ "host", "port", "transport" ],
+      "properties" : {
+        "host" : {
+          "description" : "Hostname to use",
+          "type" : "string"
+        },
+        "message-name" : {
+          "description" : "The name of the message to send.",
+          "type" : "string"
+        },
+        "port" : {
+          "description" : "Port number to use",
+          "type" : "integer"
+        },
+        "transport" : {
+          "description" : "Transport to use, can be either http or netty",
+          "enum" : [ "http", "netty" ],
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "protocol" : {
+          "description" : "Avro protocol to use",
+          "type" : "string"
+        },
+        "protocol-class-name" : {
+          "description" : "Avro protocol to use defined by the FQN class name",
+          "type" : "string"
+        },
+        "protocol-location" : {
+          "description" : "Avro protocol location",
+          "type" : "string"
+        },
+        "reflection-protocol" : {
+          "description" : "If protocol object provided is reflection protocol. Should be used only with protocol parameter because for protocolClassName protocol type will be auto detected",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "single-parameter" : {
+          "description" : "If true, consumer parameter won't be wrapped into array. Will fail if protocol specifies more then 1 parameter for the message",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "uri-authority" : {
+          "description" : "Authority to use (username and password)",
+          "type" : "string"
+        }
+      }
+    },
+    "aws-cw" : {
+      "type" : "object",
+      "required" : [ "namespace" ],
+      "properties" : {
+        "namespace" : {
+          "description" : "The metric namespace",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-cw-client" : {
+          "description" : "To use the AmazonCloudWatch as the client",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "name" : {
+          "description" : "The metric name",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the CW client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the CW client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the CW client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which CW client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "timestamp" : {
+          "description" : "The metric timestamp",
+          "type" : "string"
+        },
+        "unit" : {
+          "description" : "The metric unit",
+          "type" : "string"
+        },
+        "value" : {
+          "description" : "The metric value",
+          "type" : "number"
+        }
+      }
+    },
+    "aws-ddb" : {
+      "type" : "object",
+      "required" : [ "tableName" ],
+      "properties" : {
+        "table-name" : {
+          "description" : "The name of the table currently worked with.",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-ddb-client" : {
+          "description" : "To use the AmazonDynamoDB as the client",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "consistent-read" : {
+          "description" : "Determines whether or not strong consistency should be enforced when data is read.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "key-attribute-name" : {
+          "description" : "Attribute name when creating table",
+          "type" : "string"
+        },
+        "key-attribute-type" : {
+          "description" : "Attribute type when creating table",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "What operation to perform",
+          "default" : "PutItem",
+          "enum" : [ "BatchGetItems", "DeleteItem", "DeleteTable", "DescribeTable", "GetItem", "PutItem", "Query", "Scan", "UpdateItem", "UpdateTable" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the DDB client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the DDB client. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the DDB client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "read-capacity" : {
+          "description" : "The provisioned throughput to reserve for reading resources from your table",
+          "type" : "integer"
+        },
+        "region" : {
+          "description" : "The region in which DDB client needs to work",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "write-capacity" : {
+          "description" : "The provisioned throughput to reserved for writing resources to your table",
+          "type" : "integer"
+        }
+      }
+    },
+    "aws-ddbstream" : {
+      "type" : "object",
+      "required" : [ "tableName" ],
+      "properties" : {
+        "table-name" : {
+          "description" : "Name of the dynamodb table",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-dynamo-db-streams-client" : {
+          "description" : "Amazon DynamoDB client to use for all requests for this endpoint",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "iterator-type" : {
+          "description" : "Defines where in the DynaboDB stream to start getting records. Note that using TRIM_HORIZON can cause a significant delay before the stream has caught up to real-time. if {AT,AFTER}_SEQUENCE_NUMBER are used, then a sequenceNumberProvider MUST be supplied.",
+          "default" : "LATEST",
+          "enum" : [ "TRIM_HORIZON", "LATEST", "AT_SEQUENCE_NUMBER", "AFTER_SEQUENCE_NUMBER" ],
+          "type" : "string"
+        },
+        "max-results-per-request" : {
+          "description" : "Maximum number of records that will be fetched in each poll",
+          "type" : "integer"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the DDBStreams client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the DDBStreams client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the DDBStreams client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which DDBStreams client needs to work",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "sequence-number-provider" : {
+          "description" : "Provider for the sequence number when using one of the two ShardIteratorType.{AT,AFTER}_SEQUENCE_NUMBER iterator types. Can be a registry reference or a literal sequence number.",
+          "type" : "string"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-ec2" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-ec2-client" : {
+          "description" : "To use a existing configured AmazonEC2Client as client",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform. It can be createAndRunInstances, startInstances, stopInstances, terminateInstances, describeInstances, describeInstancesStatus, rebootInstances, monitorInstances, unmonitorInstances, createTags or deleteTags",
+          "enum" : [ "createAndRunInstances", "startInstances", "stopInstances", "terminateInstances", "describeInstances", "describeInstancesStatus", "rebootInstances", "monitorInstances", "unmonitorInstances", "createTags", "deleteTags" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the EC2 client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the EC2 client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the EC2 client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which ECS client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-ecs" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "ecs-client" : {
+          "description" : "To use a existing configured AWS ECS as client",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "enum" : [ "listClusters", "describeCluster", "createCluster", "deleteCluster" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the ECS client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the ECS client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the ECS client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which ECS client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-eks" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "eks-client" : {
+          "description" : "To use a existing configured AWS EKS as client",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "enum" : [ "listClusters", "describeCluster", "createCluster", "deleteCluster" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the EKS client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the EKS client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the EKS client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which EKS client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-iam" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "iam-client" : {
+          "description" : "To use a existing configured AWS IAM as client",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "enum" : [ "listAccessKeys", "createUser", "deleteUser", "getUser", "listUsers", "createAccessKey", "deleteAccessKey", "updateAccessKey", "createGroup", "deleteGroup", "listGroups", "addUserToGroup", "removeUserFromGroup" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the IAM client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the IAM client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the IAM client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which IAM client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-kinesis" : {
+      "type" : "object",
+      "required" : [ "streamName" ],
+      "properties" : {
+        "stream-name" : {
+          "description" : "Name of the stream",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-kinesis-client" : {
+          "description" : "Amazon Kinesis client to use for all requests for this endpoint",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "iterator-type" : {
+          "description" : "Defines where in the Kinesis stream to start getting records",
+          "default" : "TRIM_HORIZON",
+          "enum" : [ "AT_SEQUENCE_NUMBER", "AFTER_SEQUENCE_NUMBER", "TRIM_HORIZON", "LATEST", "AT_TIMESTAMP" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "max-results-per-request" : {
+          "description" : "Maximum number of records that will be fetched in each poll",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the Kinesis client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the Kinesis client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the Kinesis client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which Kinesis client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1)You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "sequence-number" : {
+          "description" : "The sequence number to start polling from. Required if iteratorType is set to AFTER_SEQUENCE_NUMBER or AT_SEQUENCE_NUMBER",
+          "type" : "string"
+        },
+        "shard-closed" : {
+          "description" : "Define what will be the behavior in case of shard closed. Possible value are ignore, silent and fail. In case of ignore a message will be logged and the consumer will restart from the beginning,in case of silent there will be no logging and the consumer will start from the beginning,in case of fail a ReachedClosedStateException will be raised",
+          "default" : "ignore",
+          "enum" : [ "ignore", "fail", "silent" ],
+          "type" : "string"
+        },
+        "shard-id" : {
+          "description" : "Defines which shardId in the Kinesis stream to get records from",
+          "type" : "string"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-kinesis-firehose" : {
+      "type" : "object",
+      "required" : [ "streamName" ],
+      "properties" : {
+        "stream-name" : {
+          "description" : "Name of the stream",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-kinesis-firehose-client" : {
+          "description" : "Amazon Kinesis Firehose client to use for all requests for this endpoint",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the Kinesis Firehose client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the Kinesis Firehose client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the Kinesis Firehose client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which Kinesis client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1)You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-kms" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "kms-client" : {
+          "description" : "To use a existing configured AWS KMS as client",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "enum" : [ "listKeys", "createKey", "disableKey", "scheduleKeyDeletion", "describeKey", "enableKey" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the KMS client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the KMS client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the KMS client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which KMS client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-lambda" : {
+      "type" : "object",
+      "required" : [ "function" ],
+      "properties" : {
+        "function" : {
+          "description" : "Name of the Lambda function.",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "aws-lambda-client" : {
+          "description" : "To use a existing configured AwsLambdaClient as client",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform. It can be listFunctions, getFunction, createFunction, deleteFunction or invokeFunction",
+          "default" : "invokeFunction",
+          "enum" : [ "listFunctions", "getFunction", "createAlias", "deleteAlias", "getAlias", "listAliases", "createFunction", "deleteFunction", "invokeFunction", "updateFunction", "createEventSourceMapping", "deleteEventSourceMapping", "listEventSourceMapping", "listTags", "tagResource", "untagResource", "publishVersion", "listVersions" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the Lambda client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the Lambda client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the Lambda client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "Amazon AWS Region. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-mq" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-mq-client" : {
+          "description" : "To use a existing configured AmazonMQClient as client",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform. It can be listBrokers,createBroker,deleteBroker",
+          "enum" : [ "listBrokers", "createBroker", "deleteBroker", "rebootBroker", "updateBroker", "describeBroker" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the MQ client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the MQ client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the MQ client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which MQ client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-msk" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "msk-client" : {
+          "description" : "To use a existing configured AWS MSK as client",
+          "type" : "string"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "enum" : [ "listClusters", "createCluster", "deleteCluster", "describeCluster" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the MSK client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the MSK client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the MSK client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which MSK client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-s3" : {
+      "type" : "object",
+      "required" : [ "bucketNameOrArn" ],
+      "properties" : {
+        "bucket-name-or-arn" : {
+          "description" : "Bucket name or ARN",
+          "type" : "string"
+        },
+        "accelerate-mode-enabled" : {
+          "description" : "Define if Accelerate Mode enabled is true or false",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-s3-client" : {
+          "description" : "Reference to a com.amazonaws.services.s3.AmazonS3 in the registry.",
+          "type" : "string"
+        },
+        "auto-create-bucket" : {
+          "description" : "Setting the autocreation of the bucket",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "autoclose-body" : {
+          "description" : "If this option is true and includeBody is true, then the S3Object.close() method will be called on exchange completion. This option is strongly related to includeBody option. In case of setting includeBody to true and autocloseBody to false, it will be up to the caller to close the S3Object stream. Setting autocloseBody to true, will close the S3Object stream automatically.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "aws-kms-key-id" : {
+          "description" : "Define the id of KMS key to use in case KMS is enabled",
+          "type" : "string"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "chunked-encoding-disabled" : {
+          "description" : "Define if disabled Chunked Encoding is true or false",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "delete-after-read" : {
+          "description" : "Delete objects from S3 after they have been retrieved. The delete is only performed if the Exchange is committed. If a rollback occurs, the object is not deleted. If this option is false, then the same objects will be retrieve over and over again on the polls. Therefore you need to use the Idempotent Consumer EIP in the route to filter out duplicates. You can filter using the S3Constants#BUCKET_NAME and S3Constants#KEY headers, or only the S3Constants#KEY header.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "delete-after-write" : {
+          "description" : "Delete file object after the S3 file has been uploaded",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "delimiter" : {
+          "description" : "The delimiter which is used in the com.amazonaws.services.s3.model.ListObjectsRequest to only consume objects we are interested in.",
+          "type" : "string"
+        },
+        "dualstack-enabled" : {
+          "description" : "Define if Dualstack enabled is true or false",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "encryption-materials" : {
+          "description" : "The encryption materials to use in case of Symmetric/Asymmetric client usage",
+          "type" : "string"
+        },
+        "endpoint-configuration" : {
+          "description" : "Amazon AWS Endpoint Configuration",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "file-name" : {
+          "description" : "To get the object from the bucket with the given file name",
+          "type" : "string"
+        },
+        "force-global-bucket-access-enabled" : {
+          "description" : "Define if Force Global Bucket Access enabled is true or false",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "include-body" : {
+          "description" : "If it is true, the exchange body will be set to a stream to the contents of the file. If false, the headers will be set with the S3 object metadata, but the body will be null. This option is strongly related to autocloseBody option. In case of setting includeBody to true and autocloseBody to false, it will be up to the caller to close the S3Object stream. Setting autocloseBody to true, will close the S3Object stream automatically.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "key-name" : {
+          "description" : "Setting the key name for an element in the bucket through endpoint parameter",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "max-connections" : {
+          "description" : "Set the maxConnections parameter in the S3 client configuration",
+          "default" : "60",
+          "type" : "integer"
+        },
+        "max-messages-per-poll" : {
+          "description" : "Gets the maximum number of messages as a limit to poll at each polling. Gets the maximum number of messages as a limit to poll at each polling. The default value is 10. Use 0 or a negative number to set it as unlimited.",
+          "default" : "10",
+          "type" : "integer"
+        },
+        "multi-part-upload" : {
+          "description" : "If it is true, camel will upload the file with multi part format, the part size is decided by the option of partSize",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to do in case the user don't want to do only an upload",
+          "enum" : [ "copyObject", "deleteBucket", "listBuckets", "downloadLink" ],
+          "type" : "string"
+        },
+        "part-size" : {
+          "description" : "Setup the partSize which is used in multi part upload, the default size is 25M.",
+          "default" : "26214400",
+          "type" : "integer"
+        },
+        "path-style-access" : {
+          "description" : "Whether or not the S3 client should use path style access",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "payload-signing-enabled" : {
+          "description" : "Define if Payload Signing enabled is true or false",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "policy" : {
+          "description" : "The policy for this queue to set in the com.amazonaws.services.s3.AmazonS3#setBucketPolicy() method.",
+          "type" : "string"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "prefix" : {
+          "description" : "The prefix which is used in the com.amazonaws.services.s3.model.ListObjectsRequest to only consume objects we are interested in.",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the S3 client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "Specify a proxy port to be used inside the client definition.",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the S3 client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which S3 client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "server-side-encryption" : {
+          "description" : "Sets the server-side encryption algorithm when encrypting the object using AWS-managed keys. For example use AES256.",
+          "type" : "string"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "storage-class" : {
+          "description" : "The storage class to set in the com.amazonaws.services.s3.model.PutObjectRequest request.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "use-aws-kms" : {
+          "description" : "Define if KMS must be used or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "use-encryption" : {
+          "description" : "Define if encryption must be used or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "use-iam-credentials" : {
+          "description" : "Set whether the S3 client should expect to load credentials on an EC2 instance or to expect static credentials to be passed in.",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-sdb" : {
+      "type" : "object",
+      "required" : [ "domainName" ],
+      "properties" : {
+        "domain-name" : {
+          "description" : "The name of the domain currently worked with.",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-sdb-client" : {
+          "description" : "To use the AmazonSimpleDB as the client",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "consistent-read" : {
+          "description" : "Determines whether or not strong consistency should be enforced when data is read.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "max-number-of-domains" : {
+          "description" : "The maximum number of domain names you want returned. The range is 1 to 100.",
+          "type" : "integer"
+        },
+        "operation" : {
+          "description" : "Operation to perform",
+          "default" : "PutAttributes",
+          "enum" : [ "BatchDeleteAttributes", "BatchPutAttributes", "DeleteAttributes", "DeleteDomain", "DomainMetadata", "GetAttributes", "ListDomains", "PutAttributes", "Select" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the SDB client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the SDB client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the SDB client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which SDB client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-ses" : {
+      "type" : "object",
+      "required" : [ "from" ],
+      "properties" : {
+        "from" : {
+          "description" : "The sender's email address.",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-ses-client" : {
+          "description" : "To use the AmazonSimpleEmailService as the client",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the SES client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the SES client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the SES client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which SES client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "reply-to-addresses" : {
+          "description" : "List of reply-to email address(es) for the message, override it using 'CamelAwsSesReplyToAddresses' header.",
+          "type" : "string"
+        },
+        "return-path" : {
+          "description" : "The email address to which bounce notifications are to be forwarded, override it using 'CamelAwsSesReturnPath' header.",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "subject" : {
+          "description" : "The subject which is used if the message header 'CamelAwsSesSubject' is not present.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "to" : {
+          "description" : "List of destination email address. Can be overriden with 'CamelAwsSesTo' header.",
+          "type" : "string"
+        }
+      }
+    },
+    "aws-sns" : {
+      "type" : "object",
+      "required" : [ "topicNameOrArn" ],
+      "properties" : {
+        "topic-name-or-arn" : {
+          "description" : "Topic name or ARN",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-sns-client" : {
+          "description" : "To use the AmazonSNS as the client",
+          "type" : "string"
+        },
+        "amazon-sqs-client" : {
+          "description" : "An SQS Client to use as bridge between SNS and SQS",
+          "type" : "string"
+        },
+        "auto-create-topic" : {
+          "description" : "Setting the autocreation of the topic",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "header-filter-strategy" : {
+          "description" : "To use a custom HeaderFilterStrategy to map headers to/from Camel.",
+          "type" : "string"
+        },
+        "kms-master-key-id" : {
+          "description" : "The ID of an AWS-managed customer master key (CMK) for Amazon SNS or a custom CMK.",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "message-structure" : {
+          "description" : "The message structure to use such as json",
+          "type" : "string"
+        },
+        "policy" : {
+          "description" : "The policy for this queue",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the SNS client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the SNS client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the SNS client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "queue-url" : {
+          "description" : "The queueUrl to subscribe to",
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which SNS client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "server-side-encryption-enabled" : {
+          "description" : "Define if Server Side Encryption is enabled or not on the topic",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "subject" : {
+          "description" : "The subject which is used if the message header 'CamelAwsSnsSubject' is not present.",
+          "type" : "string"
+        },
+        "subscribe-sn-sto-sqs" : {
+          "description" : "Define if the subscription between SNS Topic and SQS must be done or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws-sqs" : {
+      "type" : "object",
+      "required" : [ "queueNameOrArn" ],
+      "properties" : {
+        "queue-name-or-arn" : {
+          "description" : "Queue name or ARN",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-aws-host" : {
+          "description" : "The hostname of the Amazon AWS cloud.",
+          "default" : "amazonaws.com",
+          "type" : "string"
+        },
+        "amazon-sqs-client" : {
+          "description" : "To use the AmazonSQS as client",
+          "type" : "string"
+        },
+        "attribute-names" : {
+          "description" : "A list of attribute names to receive when consuming. Multiple names can be separated by comma.",
+          "type" : "string"
+        },
+        "auto-create-queue" : {
+          "description" : "Setting the autocreation of the queue",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "concurrent-consumers" : {
+          "description" : "Allows you to use multiple threads to poll the sqs queue to increase throughput",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "default-visibility-timeout" : {
+          "description" : "The default visibility timeout (in seconds)",
+          "type" : "integer"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "delay-queue" : {
+          "description" : "Define if you want to apply delaySeconds option to the queue or on single messages",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "delay-seconds" : {
+          "description" : "Delay sending messages for a number of seconds.",
+          "type" : "integer"
+        },
+        "delete-after-read" : {
+          "description" : "Delete message from SQS after it has been read",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "delete-if-filtered" : {
+          "description" : "Whether or not to send the DeleteMessage to the SQS queue if an exchange fails to get through a filter. If 'false' and exchange does not make it through a Camel filter upstream in the route, then don't send DeleteMessage.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "extend-message-visibility" : {
+          "description" : "If enabled then a scheduled background task will keep extending the message visibility on SQS. This is needed if it takes a long time to process the message. If set to true defaultVisibilityTimeout must be set. See details at Amazon docs.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "header-filter-strategy" : {
+          "description" : "To use a custom HeaderFilterStrategy to map headers to/from Camel.",
+          "type" : "string"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "kms-data-key-reuse-period-seconds" : {
+          "description" : "The length of time, in seconds, for which Amazon SQS can reuse a data key to encrypt or decrypt messages before calling AWS KMS again. An integer representing seconds, between 60 seconds (1 minute) and 86,400 seconds (24 hours). Default: 300 (5 minutes).",
+          "type" : "integer"
+        },
+        "kms-master-key-id" : {
+          "description" : "The ID of an AWS-managed customer master key (CMK) for Amazon SQS or a custom CMK.",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "max-messages-per-poll" : {
+          "description" : "Gets the maximum number of messages as a limit to poll at each polling. Is default unlimited, but use 0 or negative number to disable it as unlimited.",
+          "type" : "integer"
+        },
+        "maximum-message-size" : {
+          "description" : "The maximumMessageSize (in bytes) an SQS message can contain for this queue.",
+          "type" : "integer"
+        },
+        "message-attribute-names" : {
+          "description" : "A list of message attribute names to receive when consuming. Multiple names can be separated by comma.",
+          "type" : "string"
+        },
+        "message-deduplication-id-strategy" : {
+          "description" : "Only for FIFO queues. Strategy for setting the messageDeduplicationId on the message. Can be one of the following options: useExchangeId, useContentBasedDeduplication. For the useContentBasedDeduplication option, no messageDeduplicationId will be set on the message.",
+          "default" : "useExchangeId",
+          "enum" : [ "useExchangeId", "useContentBasedDeduplication" ],
+          "type" : "string"
+        },
+        "message-group-id-strategy" : {
+          "description" : "Only for FIFO queues. Strategy for setting the messageGroupId on the message. Can be one of the following options: useConstant, useExchangeId, usePropertyValue. For the usePropertyValue option, the value of property CamelAwsMessageGroupId will be used.",
+          "enum" : [ "useConstant", "useExchangeId", "usePropertyValue" ],
+          "type" : "string"
+        },
+        "message-retention-period" : {
+          "description" : "The messageRetentionPeriod (in seconds) a message will be retained by SQS for this queue.",
+          "type" : "integer"
+        },
+        "operation" : {
+          "description" : "The operation to do in case the user don't want to send only a message",
+          "enum" : [ "sendBatchMessage", "deleteMessage", "listQueues" ],
+          "type" : "string"
+        },
+        "policy" : {
+          "description" : "The policy for this queue",
+          "type" : "string"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "protocol" : {
+          "description" : "The underlying protocol used to communicate with SQS",
+          "default" : "https",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the SQS client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the SQS client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the SQS client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "queue-owner-aws-account-id" : {
+          "description" : "Specify the queue owner aws account id when you need to connect the queue with different account owner.",
+          "type" : "string"
+        },
+        "queue-url" : {
+          "description" : "To define the queueUrl explicitly. All other parameters, which would influence the queueUrl, are ignored. This parameter is intended to be used, to connect to a mock implementation of SQS, for testing purposes.",
+          "type" : "string"
+        },
+        "receive-message-wait-time-seconds" : {
+          "description" : "If you do not specify WaitTimeSeconds in the request, the queue attribute ReceiveMessageWaitTimeSeconds is used to determine how long to wait.",
+          "type" : "integer"
+        },
+        "redrive-policy" : {
+          "description" : "Specify the policy that send message to DeadLetter queue. See detail at Amazon docs.",
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "Specify the queue region which could be used with queueOwnerAWSAccountId to build the service URL. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "server-side-encryption-enabled" : {
+          "description" : "Define if Server Side Encryption is enabled or not on the queue",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "visibility-timeout" : {
+          "description" : "The duration (in seconds) that the received messages are hidden from subsequent retrieve requests after being retrieved by a ReceiveMessage request to set in the com.amazonaws.services.sqs.model.SetQueueAttributesRequest. This only make sense if its different from defaultVisibilityTimeout. It changes the queue visibility timeout attribute permanently.",
+          "type" : "integer"
+        },
+        "wait-time-seconds" : {
+          "description" : "Duration in seconds (0 to 20) that the ReceiveMessage action call will wait until a message is in the queue to include in the response.",
+          "type" : "integer"
+        }
+      }
+    },
+    "aws-swf" : {
+      "type" : "object",
+      "required" : [ "type" ],
+      "properties" : {
+        "type" : {
+          "description" : "Activity or workflow",
+          "enum" : [ "activity", "workflow" ],
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key.",
+          "type" : "string"
+        },
+        "activity-list" : {
+          "description" : "The list name to consume activities from.",
+          "type" : "string"
+        },
+        "activity-scheduling-options" : {
+          "description" : "Activity scheduling options",
+          "type" : "string"
+        },
+        "activity-thread-pool-size" : {
+          "description" : "Maximum number of threads in work pool for activity.",
+          "default" : "100",
+          "type" : "integer"
+        },
+        "activity-type-execution-options" : {
+          "description" : "Activity execution options",
+          "type" : "string"
+        },
+        "activity-type-registration-options" : {
+          "description" : "Activity registration options",
+          "type" : "string"
+        },
+        "amazon-sw-client" : {
+          "description" : "To use the given AmazonSimpleWorkflowClient as client",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "child-policy" : {
+          "description" : "The policy to use on child workflows when terminating a workflow.",
+          "type" : "string"
+        },
+        "client-configuration-parameters" : {
+          "description" : "To configure the ClientConfiguration using the key/values from the Map.",
+          "type" : "string"
+        },
+        "data-converter" : {
+          "description" : "An instance of com.amazonaws.services.simpleworkflow.flow.DataConverter to use for serializing/deserializing the data.",
+          "type" : "string"
+        },
+        "domain-name" : {
+          "description" : "The workflow domain to use.",
+          "type" : "string"
+        },
+        "event-name" : {
+          "description" : "The workflow or activity event name to use.",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "execution-start-to-close-timeout" : {
+          "description" : "Set the execution start to close timeout.",
+          "default" : "3600",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "Workflow operation",
+          "default" : "START",
+          "enum" : [ "SIGNAL", "CANCEL", "TERMINATE", "GET_STATE", "START", "DESCRIBE", "GET_HISTORY" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "Amazon AWS Region. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "s-w-client-parameters" : {
+          "description" : "To configure the AmazonSimpleWorkflowClient using the key/values from the Map.",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key.",
+          "type" : "string"
+        },
+        "signal-name" : {
+          "description" : "The name of the signal to send to the workflow.",
+          "type" : "string"
+        },
+        "start-workflow-options-parameters" : {
+          "description" : "To configure the StartWorkflowOptions using the key/values from the Map.",
+          "type" : "string"
+        },
+        "state-result-type" : {
+          "description" : "The type of the result when a workflow state is queried.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "task-start-to-close-timeout" : {
+          "description" : "Set the task start to close timeout.",
+          "default" : "600",
+          "type" : "string"
+        },
+        "termination-details" : {
+          "description" : "Details for terminating a workflow.",
+          "type" : "string"
+        },
+        "termination-reason" : {
+          "description" : "The reason for terminating a workflow.",
+          "type" : "string"
+        },
+        "version" : {
+          "description" : "The workflow or activity event version to use.",
+          "type" : "string"
+        },
+        "workflow-list" : {
+          "description" : "The list name to consume workflows from.",
+          "type" : "string"
+        },
+        "workflow-type-registration-options" : {
+          "description" : "Workflow registration options",
+          "type" : "string"
+        }
+      }
+    },
+    "aws-translate" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "autodetect-source-language" : {
+          "description" : "Being able to autodetect the source language",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "default" : "translateText",
+          "enum" : [ "translateText" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the Translate client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the Translate client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the Translate client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which Translate client needs to work. When using this parameter, the configuration will expect the capitalized name of the region (for example AP_EAST_1) You'll need to use the name Regions.EU_WEST_1.name()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "source-language" : {
+          "description" : "Source language to use",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "target-language" : {
+          "description" : "Target language to use",
+          "type" : "string"
+        },
+        "translate-client" : {
+          "description" : "To use a existing configured AWS Translate as client",
+          "type" : "string"
+        }
+      }
+    },
+    "aws2-athena" : {
+      "type" : "object",
+      "required" : [ "label" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key.",
+          "type" : "string"
+        },
+        "amazon-athena-client" : {
+          "description" : "The AmazonAthena instance to use as the client.",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "client-request-token" : {
+          "description" : "A unique string to ensure issues queries are idempotent. It is unlikely you will need to set this.",
+          "type" : "string"
+        },
+        "database" : {
+          "description" : "The Athena database to use.",
+          "type" : "string"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll for query execution status. See the section 'Waiting for Query Completion and Retrying Failed Queries' to learn more.",
+          "default" : "2000",
+          "type" : "integer"
+        },
+        "encryption-option" : {
+          "description" : "The encryption type to use when storing query results in S3. One of SSE_S3, SSE_KMS, or CSE_KMS.",
+          "enum" : [ "SSE_S3", "SSE_KMS", "CSE_KMS", "null" ],
+          "type" : "string"
+        },
+        "include-trace" : {
+          "description" : "Include useful trace information at the beginning of queries as an SQL comment (prefixed with --).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll for query execution status. See the section 'Waiting for Query Completion and Retrying Failed Queries' to learn more.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "kms-key" : {
+          "description" : "For SSE-KMS and CSE-KMS, this is the KMS key ARN or ID.",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "max-attempts" : {
+          "description" : "Maximum number of times to attempt a query. Set to 1 to disable retries. See the section 'Waiting for Query Completion and Retrying Failed Queries' to learn more.",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "max-results" : {
+          "description" : "Max number of results to return for the given operation (if supported by the Athena API endpoint). If not set, will use the Athena API default for the given operation.",
+          "type" : "integer"
+        },
+        "next-token" : {
+          "description" : "Pagination token to use in the case where the response from the previous request was truncated.",
+          "type" : "string"
+        },
+        "operation" : {
+          "description" : "The Athena API function to call.",
+          "default" : "startQueryExecution",
+          "enum" : [ "getQueryExecution", "getQueryResults", "listQueryExecutions", "startQueryExecution" ],
+          "type" : "string"
+        },
+        "output-location" : {
+          "description" : "The location in Amazon S3 where query results are stored, such as s3://path/to/query/bucket/. Ensure this value ends with a forward slash ('/').",
+          "type" : "string"
+        },
+        "output-type" : {
+          "description" : "How query results should be returned. One of StreamList (default - return a GetQueryResultsIterable that can page through all results), SelectList (returns at most 1,000 rows at a time, plus a NextToken value as a header than can be used for manual pagination of results), S3Pointer (return an S3 path pointing to the results).",
+          "default" : "StreamList",
+          "enum" : [ "StreamList", "SelectList", "S3Pointer" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the Athena client.",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the Athena client.",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the Athena client.",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "query-execution-id" : {
+          "description" : "The unique ID identifying the query execution.",
+          "type" : "string"
+        },
+        "query-string" : {
+          "description" : "The SQL query to run. Except for simple queries, prefer setting this as the body of the Exchange or as a header using Athena2Constants.QUERY_STRING to avoid having to deal with URL encoding issues.",
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which Athena client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1). You'll need to use the name Region.EU_WEST_1.id().",
+          "type" : "string"
+        },
+        "reset-wait-timeout-on-retry" : {
+          "description" : "Reset the waitTimeout countdown in the event of a query retry. If set to true, potential max time spent waiting for queries is equal to waitTimeout x maxAttempts. See the section 'Waiting for Query Completion and Retrying Failed Queries' to learn more.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "retry" : {
+          "description" : "Optional comma separated list of error types to retry the query for. Use 'retryable' to retry all retryable failure conditions (e.g. generic errors and resources exhausted), 'generic' to retry 'GENERIC_INTERNAL_ERROR' failures, 'exhausted' to retry queries that have exhausted resource limits, 'always' to always retry regardless of failure condition, or 'never' or null to never retry (default). See the section 'Waiting for Query Completion and Retrying Failed Qu [...]
+          "default" : "never",
+          "enum" : [ "never", "always", "retryable", "exhausted", "generic" ],
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "wait-timeout" : {
+          "description" : "Optional max wait time in millis to wait for a successful query completion. See the section 'Waiting for Query Completion and Retrying Failed Queries' to learn more.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "work-group" : {
+          "description" : "The workgroup to use for running the query.",
+          "type" : "string"
+        }
+      }
+    },
+    "aws2-cw" : {
+      "type" : "object",
+      "required" : [ "namespace" ],
+      "properties" : {
+        "namespace" : {
+          "description" : "The metric namespace",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-cw-client" : {
+          "description" : "To use the AmazonCloudWatch as the client",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "name" : {
+          "description" : "The metric name",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the CW client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the CW client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the CW client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which EKS client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "timestamp" : {
+          "description" : "The metric timestamp",
+          "type" : "string"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "unit" : {
+          "description" : "The metric unit",
+          "type" : "string"
+        },
+        "value" : {
+          "description" : "The metric value",
+          "type" : "number"
+        }
+      }
+    },
+    "aws2-ddb" : {
+      "type" : "object",
+      "required" : [ "tableName" ],
+      "properties" : {
+        "table-name" : {
+          "description" : "The name of the table currently worked with.",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-ddb-client" : {
+          "description" : "To use the AmazonDynamoDB as the client",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "consistent-read" : {
+          "description" : "Determines whether or not strong consistency should be enforced when data is read.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "key-attribute-name" : {
+          "description" : "Attribute name when creating table",
+          "type" : "string"
+        },
+        "key-attribute-type" : {
+          "description" : "Attribute type when creating table",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "What operation to perform",
+          "default" : "PutItem",
+          "enum" : [ "BatchGetItems", "DeleteItem", "DeleteTable", "DescribeTable", "GetItem", "PutItem", "Query", "Scan", "UpdateItem", "UpdateTable" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the DDB client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "The region in which DynamoDB client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the DDB client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "read-capacity" : {
+          "description" : "The provisioned throughput to reserve for reading resources from your table",
+          "type" : "integer"
+        },
+        "region" : {
+          "description" : "The region in which DDB client needs to work",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "write-capacity" : {
+          "description" : "The provisioned throughput to reserved for writing resources to your table",
+          "type" : "integer"
+        }
+      }
+    },
+    "aws2-ddbstream" : {
+      "type" : "object",
+      "required" : [ "tableName" ],
+      "properties" : {
+        "table-name" : {
+          "description" : "Name of the dynamodb table",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-dynamo-db-streams-client" : {
+          "description" : "Amazon DynamoDB client to use for all requests for this endpoint",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "iterator-type" : {
+          "description" : "Defines where in the DynaboDB stream to start getting records. Note that using TRIM_HORIZON can cause a significant delay before the stream has caught up to real-time. if {AT,AFTER}_SEQUENCE_NUMBER are used, then a sequenceNumberProvider MUST be supplied.",
+          "default" : "LATEST",
+          "enum" : [ "TRIM_HORIZON", "LATEST", "AT_SEQUENCE_NUMBER", "AFTER_SEQUENCE_NUMBER", "null" ],
+          "type" : "string"
+        },
+        "max-results-per-request" : {
+          "description" : "Maximum number of records that will be fetched in each poll",
+          "type" : "integer"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the DDBStreams client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the DDBStreams client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the DDBStreams client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which DDBStreams client needs to work",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "sequence-number-provider" : {
+          "description" : "Provider for the sequence number when using one of the two ShardIteratorType.{AT,AFTER}_SEQUENCE_NUMBER iterator types. Can be a registry reference or a literal sequence number.",
+          "type" : "string"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-ec2" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-ec2-client" : {
+          "description" : "To use a existing configured AmazonEC2Client as client",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform. It can be createAndRunInstances, startInstances, stopInstances, terminateInstances, describeInstances, describeInstancesStatus, rebootInstances, monitorInstances, unmonitorInstances, createTags or deleteTags",
+          "enum" : [ "createAndRunInstances", "startInstances", "stopInstances", "terminateInstances", "describeInstances", "describeInstancesStatus", "rebootInstances", "monitorInstances", "unmonitorInstances", "createTags", "deleteTags" ],
+          "type" : "string"
+        },
+        "pojo-request" : {
+          "description" : "If we want to use a POJO request as body or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the EC2 client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the EC2 client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the EC2 client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which EC2 client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-ecs" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "ecs-client" : {
+          "description" : "To use a existing configured AWS ECS as client",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "enum" : [ "listClusters", "describeCluster", "createCluster", "deleteCluster" ],
+          "type" : "string"
+        },
+        "pojo-request" : {
+          "description" : "If we want to use a POJO request as body or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the ECS client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the ECS client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the ECS client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which ECS client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-eks" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "eks-client" : {
+          "description" : "To use a existing configured AWS EKS as client",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "enum" : [ "listClusters", "describeCluster", "createCluster", "deleteCluster" ],
+          "type" : "string"
+        },
+        "pojo-request" : {
+          "description" : "If we want to use a POJO request as body or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the EKS client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the EKS client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the EKS client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which EKS client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-iam" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "iam-client" : {
+          "description" : "To use a existing configured AWS IAM as client",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "enum" : [ "listAccessKeys", "createUser", "deleteUser", "getUser", "listUsers", "createAccessKey", "deleteAccessKey", "updateAccessKey", "createGroup", "deleteGroup", "listGroups", "addUserToGroup", "removeUserFromGroup" ],
+          "type" : "string"
+        },
+        "pojo-request" : {
+          "description" : "If we want to use a POJO request as body or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the IAM client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the IAM client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the IAM client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which IAM client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "default" : "aws-global",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-kinesis" : {
+      "type" : "object",
+      "required" : [ "streamName" ],
+      "properties" : {
+        "stream-name" : {
+          "description" : "Name of the stream",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-kinesis-client" : {
+          "description" : "Amazon Kinesis client to use for all requests for this endpoint",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "iterator-type" : {
+          "description" : "Defines where in the Kinesis stream to start getting records",
+          "default" : "TRIM_HORIZON",
+          "enum" : [ "AT_SEQUENCE_NUMBER", "AFTER_SEQUENCE_NUMBER", "TRIM_HORIZON", "LATEST", "AT_TIMESTAMP", "null" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "max-results-per-request" : {
+          "description" : "Maximum number of records that will be fetched in each poll",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the Kinesis client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the Kinesis client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the Kinesis client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which Kinesis Firehose client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "sequence-number" : {
+          "description" : "The sequence number to start polling from. Required if iteratorType is set to AFTER_SEQUENCE_NUMBER or AT_SEQUENCE_NUMBER",
+          "type" : "string"
+        },
+        "shard-closed" : {
+          "description" : "Define what will be the behavior in case of shard closed. Possible value are ignore, silent and fail. In case of ignore a message will be logged and the consumer will restart from the beginning,in case of silent there will be no logging and the consumer will start from the beginning,in case of fail a ReachedClosedStateException will be raised",
+          "default" : "ignore",
+          "enum" : [ "ignore", "fail", "silent" ],
+          "type" : "string"
+        },
+        "shard-id" : {
+          "description" : "Defines which shardId in the Kinesis stream to get records from",
+          "type" : "string"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-kinesis-firehose" : {
+      "type" : "object",
+      "required" : [ "streamName" ],
+      "properties" : {
+        "stream-name" : {
+          "description" : "Name of the stream",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-kinesis-firehose-client" : {
+          "description" : "Amazon Kinesis Firehose client to use for all requests for this endpoint",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to do in case the user don't want to send only a record",
+          "enum" : [ "sendBatchRecord", "createDeliveryStream", "deleteDeliveryStream", "updateDestination" ],
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the Kinesis Firehose client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the Kinesis Firehose client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the Kinesis Firehose client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which Kinesis Firehose client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-kms" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "kms-client" : {
+          "description" : "To use a existing configured AWS KMS as client",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "enum" : [ "listKeys", "createKey", "disableKey", "scheduleKeyDeletion", "describeKey", "enableKey" ],
+          "type" : "string"
+        },
+        "pojo-request" : {
+          "description" : "If we want to use a POJO request as body or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the KMS client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the KMS client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the KMS client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which EKS client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-lambda" : {
+      "type" : "object",
+      "required" : [ "function" ],
+      "properties" : {
+        "function" : {
+          "description" : "Name of the Lambda function.",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "aws-lambda-client" : {
+          "description" : "To use a existing configured AwsLambdaClient as client",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform. It can be listFunctions, getFunction, createFunction, deleteFunction or invokeFunction",
+          "default" : "invokeFunction",
+          "enum" : [ "listFunctions", "getFunction", "createAlias", "deleteAlias", "getAlias", "listAliases", "createFunction", "deleteFunction", "invokeFunction", "updateFunction", "createEventSourceMapping", "deleteEventSourceMapping", "listEventSourceMapping", "listTags", "tagResource", "untagResource", "publishVersion", "listVersions" ],
+          "type" : "string"
+        },
+        "pojo-request" : {
+          "description" : "If we want to use a POJO request as body or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the Lambda client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the Lambda client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the Lambda client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which ECS client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-mq" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-mq-client" : {
+          "description" : "To use a existing configured AmazonMQClient as client",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform. It can be listBrokers,createBroker,deleteBroker",
+          "enum" : [ "listBrokers", "createBroker", "deleteBroker", "rebootBroker", "updateBroker", "describeBroker" ],
+          "type" : "string"
+        },
+        "pojo-request" : {
+          "description" : "If we want to use a POJO request as body or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the MQ client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the MQ client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the MQ client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which MQ client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-msk" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "msk-client" : {
+          "description" : "To use a existing configured AWS MSK as client",
+          "type" : "string"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "enum" : [ "listClusters", "createCluster", "deleteCluster", "describeCluster" ],
+          "type" : "string"
+        },
+        "pojo-request" : {
+          "description" : "If we want to use a POJO request as body or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the MSK client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the MSK client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the MSK client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which MSK client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-s3" : {
+      "type" : "object",
+      "required" : [ "bucketNameOrArn" ],
+      "properties" : {
+        "bucket-name-or-arn" : {
+          "description" : "Bucket name or ARN",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-s3-client" : {
+          "description" : "Reference to a com.amazonaws.services.s3.AmazonS3 in the registry.",
+          "type" : "string"
+        },
+        "auto-create-bucket" : {
+          "description" : "Setting the autocreation of the S3 bucket bucketName. This will apply also in case of moveAfterRead option enabled and it will create the destinationBucket if it doesn't exist already.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "autoclose-body" : {
+          "description" : "If this option is true and includeBody is true, then the S3Object.close() method will be called on exchange completion. This option is strongly related to includeBody option. In case of setting includeBody to true and autocloseBody to false, it will be up to the caller to close the S3Object stream. Setting autocloseBody to true, will close the S3Object stream automatically.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "aws-kms-key-id" : {
+          "description" : "Define the id of KMS key to use in case KMS is enabled",
+          "type" : "string"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "customer-algorithm" : {
+          "description" : "Define the customer algorithm to use in case CustomerKey is enabled",
+          "type" : "string"
+        },
+        "customer-key-id" : {
+          "description" : "Define the id of Customer key to use in case CustomerKey is enabled",
+          "type" : "string"
+        },
+        "customer-key-md5" : {
+          "description" : "Define the MD5 of Customer key to use in case CustomerKey is enabled",
+          "type" : "string"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "delete-after-read" : {
+          "description" : "Delete objects from S3 after they have been retrieved. The delete is only performed if the Exchange is committed. If a rollback occurs, the object is not deleted. If this option is false, then the same objects will be retrieve over and over again on the polls. Therefore you need to use the Idempotent Consumer EIP in the route to filter out duplicates. You can filter using the AWS2S3Constants#BUCKET_NAME and AWS2S3Constants#KEY headers, or only the AWS2S3Constan [...]
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "delete-after-write" : {
+          "description" : "Delete file object after the S3 file has been uploaded",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "delimiter" : {
+          "description" : "The delimiter which is used in the com.amazonaws.services.s3.model.ListObjectsRequest to only consume objects we are interested in.",
+          "type" : "string"
+        },
+        "destination-bucket" : {
+          "description" : "Define the destination bucket where an object must be moved when moveAfterRead is set to true.",
+          "type" : "string"
+        },
+        "destination-bucket-prefix" : {
+          "description" : "Define the destination bucket prefix to use when an object must be moved and moveAfterRead is set to true.",
+          "type" : "string"
+        },
+        "destination-bucket-suffix" : {
+          "description" : "Define the destination bucket suffix to use when an object must be moved and moveAfterRead is set to true.",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "file-name" : {
+          "description" : "To get the object from the bucket with the given file name",
+          "type" : "string"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "include-body" : {
+          "description" : "If it is true, the exchange body will be set to a stream to the contents of the file. If false, the headers will be set with the S3 object metadata, but the body will be null. This option is strongly related to autocloseBody option. In case of setting includeBody to true and autocloseBody to false, it will be up to the caller to close the S3Object stream. Setting autocloseBody to true, will close the S3Object stream automatically.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "include-folders" : {
+          "description" : "If it is true, the folders/directories will be consumed. If it is false, they will be ignored, and Exchanges will not be created for those",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "key-name" : {
+          "description" : "Setting the key name for an element in the bucket through endpoint parameter",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "max-connections" : {
+          "description" : "Set the maxConnections parameter in the S3 client configuration",
+          "default" : "60",
+          "type" : "integer"
+        },
+        "max-messages-per-poll" : {
+          "description" : "Gets the maximum number of messages as a limit to poll at each polling. Gets the maximum number of messages as a limit to poll at each polling. The default value is 10. Use 0 or a negative number to set it as unlimited.",
+          "default" : "10",
+          "type" : "integer"
+        },
+        "move-after-read" : {
+          "description" : "Move objects from S3 bucket to a different bucket after they have been retrieved. To accomplish the operation the destinationBucket option must be set. The copy bucket operation is only performed if the Exchange is committed. If a rollback occurs, the object is not moved.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "multi-part-upload" : {
+          "description" : "If it is true, camel will upload the file with multi part format, the part size is decided by the option of partSize",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to do in case the user don't want to do only an upload",
+          "enum" : [ "copyObject", "listObjects", "deleteObject", "deleteBucket", "listBuckets", "getObject", "getObjectRange" ],
+          "type" : "string"
+        },
+        "override-endpoint" : {
+          "description" : "Set the need for overidding the endpoint. This option needs to be used in combination with uriEndpointOverride option",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "part-size" : {
+          "description" : "Setup the partSize which is used in multi part upload, the default size is 25M.",
+          "default" : "26214400",
+          "type" : "integer"
+        },
+        "pojo-request" : {
+          "description" : "If we want to use a POJO request as body or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "policy" : {
+          "description" : "The policy for this queue to set in the com.amazonaws.services.s3.AmazonS3#setBucketPolicy() method.",
+          "type" : "string"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "prefix" : {
+          "description" : "The prefix which is used in the com.amazonaws.services.s3.model.ListObjectsRequest to only consume objects we are interested in.",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the SQS client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "Specify a proxy port to be used inside the client definition.",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the S3 client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which S3 client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "storage-class" : {
+          "description" : "The storage class to set in the com.amazonaws.services.s3.model.PutObjectRequest request.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "uri-endpoint-override" : {
+          "description" : "Set the overriding uri endpoint. This option needs to be used in combination with overrideEndpoint option",
+          "type" : "string"
+        },
+        "use-aws-kms" : {
+          "description" : "Define if KMS must be used or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "use-customer-key" : {
+          "description" : "Define if Customer Key must be used or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "use-iam-credentials" : {
+          "description" : "Set whether the S3 client should expect to load credentials on an EC2 instance or to expect static credentials to be passed in.",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-ses" : {
+      "type" : "object",
+      "required" : [ "from" ],
+      "properties" : {
+        "from" : {
+          "description" : "The sender's email address.",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-ses-client" : {
+          "description" : "To use the AmazonSimpleEmailService as the client",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the SES client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the SES client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the SES client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which SES client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "reply-to-addresses" : {
+          "description" : "List of reply-to email address(es) for the message, override it using 'CamelAwsSesReplyToAddresses' header.",
+          "type" : "string"
+        },
+        "return-path" : {
+          "description" : "The email address to which bounce notifications are to be forwarded, override it using 'CamelAwsSesReturnPath' header.",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "subject" : {
+          "description" : "The subject which is used if the message header 'CamelAwsSesSubject' is not present.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "to" : {
+          "description" : "List of destination email address. Can be overriden with 'CamelAwsSesTo' header.",
+          "type" : "string"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-sns" : {
+      "type" : "object",
+      "required" : [ "topicNameOrArn" ],
+      "properties" : {
+        "topic-name-or-arn" : {
+          "description" : "Topic name or ARN",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-sns-client" : {
+          "description" : "To use the AmazonSNS as the client",
+          "type" : "string"
+        },
+        "auto-create-topic" : {
+          "description" : "Setting the autocreation of the topic",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "header-filter-strategy" : {
+          "description" : "To use a custom HeaderFilterStrategy to map headers to/from Camel.",
+          "type" : "string"
+        },
+        "kms-master-key-id" : {
+          "description" : "The ID of an AWS-managed customer master key (CMK) for Amazon SNS or a custom CMK.",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "message-structure" : {
+          "description" : "The message structure to use such as json",
+          "type" : "string"
+        },
+        "policy" : {
+          "description" : "The policy for this queue",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the SNS client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the SNS client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the SNS client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "queue-url" : {
+          "description" : "The queueUrl to subscribe to",
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which SNS client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "server-side-encryption-enabled" : {
+          "description" : "Define if Server Side Encryption is enabled or not on the topic",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "subject" : {
+          "description" : "The subject which is used if the message header 'CamelAwsSnsSubject' is not present.",
+          "type" : "string"
+        },
+        "subscribe-sn-sto-sqs" : {
+          "description" : "Define if the subscription between SNS Topic and SQS must be done or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-sqs" : {
+      "type" : "object",
+      "required" : [ "queueNameOrArn" ],
+      "properties" : {
+        "queue-name-or-arn" : {
+          "description" : "Queue name or ARN",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "amazon-aws-host" : {
+          "description" : "The hostname of the Amazon AWS cloud.",
+          "default" : "amazonaws.com",
+          "type" : "string"
+        },
+        "amazon-sqs-client" : {
+          "description" : "To use the AmazonSQS as client",
+          "type" : "string"
+        },
+        "attribute-names" : {
+          "description" : "A list of attribute names to receive when consuming. Multiple names can be separated by comma.",
+          "type" : "string"
+        },
+        "auto-create-queue" : {
+          "description" : "Setting the autocreation of the queue",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "concurrent-consumers" : {
+          "description" : "Allows you to use multiple threads to poll the sqs queue to increase throughput",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "default-visibility-timeout" : {
+          "description" : "The default visibility timeout (in seconds)",
+          "type" : "integer"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "delay-queue" : {
+          "description" : "Define if you want to apply delaySeconds option to the queue or on single messages",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "delay-seconds" : {
+          "description" : "Delay sending messages for a number of seconds.",
+          "type" : "integer"
+        },
+        "delete-after-read" : {
+          "description" : "Delete message from SQS after it has been read",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "delete-if-filtered" : {
+          "description" : "Whether or not to send the DeleteMessage to the SQS queue if an exchange fails to get through a filter. If 'false' and exchange does not make it through a Camel filter upstream in the route, then don't send DeleteMessage.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "extend-message-visibility" : {
+          "description" : "If enabled then a scheduled background task will keep extending the message visibility on SQS. This is needed if it takes a long time to process the message. If set to true defaultVisibilityTimeout must be set. See details at Amazon docs.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "header-filter-strategy" : {
+          "description" : "To use a custom HeaderFilterStrategy to map headers to/from Camel.",
+          "type" : "string"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "kms-data-key-reuse-period-seconds" : {
+          "description" : "The length of time, in seconds, for which Amazon SQS can reuse a data key to encrypt or decrypt messages before calling AWS KMS again. An integer representing seconds, between 60 seconds (1 minute) and 86,400 seconds (24 hours). Default: 300 (5 minutes).",
+          "type" : "integer"
+        },
+        "kms-master-key-id" : {
+          "description" : "The ID of an AWS-managed customer master key (CMK) for Amazon SQS or a custom CMK.",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "max-messages-per-poll" : {
+          "description" : "Gets the maximum number of messages as a limit to poll at each polling. Is default unlimited, but use 0 or negative number to disable it as unlimited.",
+          "type" : "integer"
+        },
+        "maximum-message-size" : {
+          "description" : "The maximumMessageSize (in bytes) an SQS message can contain for this queue.",
+          "type" : "integer"
+        },
+        "message-attribute-names" : {
+          "description" : "A list of message attribute names to receive when consuming. Multiple names can be separated by comma.",
+          "type" : "string"
+        },
+        "message-deduplication-id-strategy" : {
+          "description" : "Only for FIFO queues. Strategy for setting the messageDeduplicationId on the message. Can be one of the following options: useExchangeId, useContentBasedDeduplication. For the useContentBasedDeduplication option, no messageDeduplicationId will be set on the message.",
+          "default" : "useExchangeId",
+          "enum" : [ "useExchangeId", "useContentBasedDeduplication" ],
+          "type" : "string"
+        },
+        "message-group-id-strategy" : {
+          "description" : "Only for FIFO queues. Strategy for setting the messageGroupId on the message. Can be one of the following options: useConstant, useExchangeId, usePropertyValue. For the usePropertyValue option, the value of property CamelAwsMessageGroupId will be used.",
+          "enum" : [ "useConstant", "useExchangeId", "usePropertyValue" ],
+          "type" : "string"
+        },
+        "message-retention-period" : {
+          "description" : "The messageRetentionPeriod (in seconds) a message will be retained by SQS for this queue.",
+          "type" : "integer"
+        },
+        "operation" : {
+          "description" : "The operation to do in case the user don't want to send only a message",
+          "enum" : [ "sendBatchMessage", "deleteMessage", "listQueues", "purgeQueue" ],
+          "type" : "string"
+        },
+        "policy" : {
+          "description" : "The policy for this queue",
+          "type" : "string"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "protocol" : {
+          "description" : "The underlying protocol used to communicate with SQS",
+          "default" : "https",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the SQS client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the SQS client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the SQS client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "queue-owner-aws-account-id" : {
+          "description" : "Specify the queue owner aws account id when you need to connect the queue with different account owner.",
+          "type" : "string"
+        },
+        "queue-url" : {
+          "description" : "To define the queueUrl explicitly. All other parameters, which would influence the queueUrl, are ignored. This parameter is intended to be used, to connect to a mock implementation of SQS, for testing purposes.",
+          "type" : "string"
+        },
+        "receive-message-wait-time-seconds" : {
+          "description" : "If you do not specify WaitTimeSeconds in the request, the queue attribute ReceiveMessageWaitTimeSeconds is used to determine how long to wait.",
+          "type" : "integer"
+        },
+        "redrive-policy" : {
+          "description" : "Specify the policy that send message to DeadLetter queue. See detail at Amazon docs.",
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which SQS client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "server-side-encryption-enabled" : {
+          "description" : "Define if Server Side Encryption is enabled or not on the queue",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "visibility-timeout" : {
+          "description" : "The duration (in seconds) that the received messages are hidden from subsequent retrieve requests after being retrieved by a ReceiveMessage request to set in the com.amazonaws.services.sqs.model.SetQueueAttributesRequest. This only make sense if its different from defaultVisibilityTimeout. It changes the queue visibility timeout attribute permanently.",
+          "type" : "integer"
+        },
+        "wait-time-seconds" : {
+          "description" : "Duration in seconds (0 to 20) that the ReceiveMessage action call will wait until a message is in the queue to include in the response.",
+          "type" : "integer"
+        }
+      }
+    },
+    "aws2-sts" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "default" : "assumeRole",
+          "enum" : [ "assumeRole", "getSessionToken", "getFederationToken" ],
+          "type" : "string"
+        },
+        "pojo-request" : {
+          "description" : "If we want to use a POJO request as body or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the STS client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the STS client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the STS client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which STS client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "default" : "aws-global",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "sts-client" : {
+          "description" : "To use a existing configured AWS STS as client",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "aws2-translate" : {
+      "type" : "object",
+      "required" : [ "label", "operation" ],
+      "properties" : {
+        "label" : {
+          "description" : "Logical name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Amazon AWS Access Key",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "autodetect-source-language" : {
+          "description" : "Being able to autodetect the source language",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "The operation to perform",
+          "default" : "translateText",
+          "enum" : [ "translateText" ],
+          "type" : "string"
+        },
+        "pojo-request" : {
+          "description" : "If we want to use a POJO request as body or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "proxy-host" : {
+          "description" : "To define a proxy host when instantiating the Translate client",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "To define a proxy port when instantiating the Translate client",
+          "type" : "integer"
+        },
+        "proxy-protocol" : {
+          "description" : "To define a proxy protocol when instantiating the Translate client",
+          "default" : "HTTPS",
+          "enum" : [ "HTTP", "HTTPS" ],
+          "type" : "string"
+        },
+        "region" : {
+          "description" : "The region in which Translate client needs to work. When using this parameter, the configuration will expect the lowercase name of the region (for example ap-east-1) You'll need to use the name Region.EU_WEST_1.id()",
+          "type" : "string"
+        },
+        "secret-key" : {
+          "description" : "Amazon AWS Secret Key",
+          "type" : "string"
+        },
+        "source-language" : {
+          "description" : "Source language to use",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "target-language" : {
+          "description" : "Target language to use",
+          "type" : "string"
+        },
+        "translate-client" : {
+          "description" : "To use a existing configured AWS Translate as client",
+          "type" : "string"
+        },
+        "trust-all-certificates" : {
+          "description" : "If we want to trust all certificates in case of overriding the endpoint",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "azure-blob" : {
+      "type" : "object",
+      "required" : [ "containerOrBlobUri" ],
+      "properties" : {
+        "container-or-blob-uri" : {
+          "description" : "Container or Blob compact Uri",
+          "type" : "string"
+        },
+        "azure-blob-client" : {
+          "description" : "The blob service client",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "blob-metadata" : {
+          "description" : "Set the blob meta-data",
+          "type" : "string"
+        },
+        "blob-offset" : {
+          "description" : "Set the blob offset for the upload or download operations, default is 0",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "blob-prefix" : {
+          "description" : "Set a prefix which can be used for listing the blobs",
+          "type" : "string"
+        },
+        "blob-type" : {
+          "description" : "Set a blob type, 'blockblob' is default",
+          "default" : "blockblob",
+          "enum" : [ "blockblob", "appendblob", "pageblob" ],
+          "type" : "string"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "close-stream-after-read" : {
+          "description" : "Close the stream after read or keep it open, default is true",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "close-stream-after-write" : {
+          "description" : "Close the stream after write or keep it open, default is true",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "credentials" : {
+          "description" : "Set the storage credentials, required in most cases",
+          "type" : "string"
+        },
+        "credentials-account-key" : {
+          "description" : "Set the storage account key used during authentication phase",
+          "type" : "string"
+        },
+        "credentials-account-name" : {
+          "description" : "Set the storage account name used during authentication phase",
+          "type" : "string"
+        },
+        "data-length" : {
+          "description" : "Set the data length for the download or page blob upload operations",
+          "type" : "integer"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "file-dir" : {
+          "description" : "Set the file directory where the downloaded blobs will be saved to",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "Blob service operation hint to the producer",
+          "default" : "listBlobs",
+          "enum" : [ "getBlob", "deleteBlob", "listBlobs", "updateBlockBlob", "uploadBlobBlocks", "commitBlobBlockList", "getBlobBlockList", "createAppendBlob", "updateAppendBlob", "createPageBlob", "updatePageBlob", "resizePageBlob", "clearPageBlob", "getPageBlobRanges" ],
+          "type" : "string"
+        },
+        "public-for-read" : {
+          "description" : "Storage resources can be public for reading their content, if this property is enabled then the credentials do not have to be set",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "stream-read-size" : {
+          "description" : "Set the minimum read size in bytes when reading the blob content",
+          "type" : "integer"
+        },
+        "stream-write-size" : {
+          "description" : "Set the size of the buffer for writing block and page blocks",
+          "type" : "integer"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "use-flat-listing" : {
+          "description" : "Specify if the flat or hierarchical blob listing should be used",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "validate-client-uri" : {
+          "description" : "Whether to validate the Azure client URI",
+          "default" : "true",
+          "type" : "boolean"
+        }
+      }
+    },
+    "azure-eventhubs" : {
+      "type" : "object",
+      "properties" : {
+        "event-hub-name" : {
+          "description" : "EventHubs name under a specific namcespace",
+          "type" : "string"
+        },
+        "namespace" : {
+          "description" : "EventHubs namespace created in Azure Portal",
+          "type" : "string"
+        },
+        "amqp-retry-options" : {
+          "description" : "Sets the retry policy for EventHubAsyncClient. If not specified, the default retry options are used.",
+          "type" : "string"
+        },
+        "amqp-transport-type" : {
+          "description" : "Sets the transport type by which all the communication with Azure Event Hubs occurs. Default value is AmqpTransportType#AMQP.",
+          "default" : "AMQP",
+          "enum" : [ "Amqp", "AmqpWebSockets" ],
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "blob-access-key" : {
+          "description" : "In case you chose the default BlobCheckpointStore, this sets access key for the associated azure account name to be used for authentication with azure blob services",
+          "type" : "string"
+        },
+        "blob-account-name" : {
+          "description" : "In case you chose the default BlobCheckpointStore, this sets Azure account name to be used for authentication with azure blob services.",
+          "type" : "string"
+        },
+        "blob-container-name" : {
+          "description" : "In case you chose the default BlobCheckpointStore, this sets the blob container that shall be used by the BlobCheckpointStore to store the checkpoint offsets",
+          "type" : "string"
+        },
+        "blob-storage-shared-key-credential" : {
+          "description" : "In case you chose the default BlobCheckpointStore, StorageSharedKeyCredential can be injected to create the azure client, this holds the important authentication information",
+          "type" : "string"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "checkpoint-store" : {
+          "description" : "Sets the CheckpointStore the EventProcessorClient will use for storing partition ownership and checkpoint information. Users can, optionally, provide their own implementation of CheckpointStore which will store ownership and checkpoint information. By default it set to use com.azure.messaging.eventhubs.checkpointstore.blob.BlobCheckpointStore which stores all checkpoint offsets into Azure Blob Storage",
+          "default" : "BlobCheckpointStore",
+          "type" : "string"
+        },
+        "connection-string" : {
+          "description" : "Instead of supplying namespace, sharedAccessKey, sharedAccessName ... etc, you can just supply the connection string for your eventHub. The connection string for EventHubs already include all the necessary information to connection to your EventHub. To learn on how to generate the connection string, take a look at this documentation: https://docs.microsoft.com/en-us/azure/event-hubs/event-hubs-get-connection-string",
+          "type" : "string"
+        },
+        "consumer-group-name" : {
+          "description" : "Sets the name of the consumer group this consumer is associated with. Events are read in the context of this group. The name of the consumer group that is created by default is {link #DEFAULT_CONSUMER_GROUP_NAME $Default}.",
+          "default" : "$Default",
+          "type" : "string"
+        },
+        "event-position" : {
+          "description" : "Sets the map containing the event position to use for each partition if a checkpoint for the partition does not exist in CheckpointStore. This map is keyed off of the partition id. If there is no checkpoint in CheckpointStore and there is no entry in this map, the processing of the partition will start from {link EventPosition#latest() latest} position.",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "partition-id" : {
+          "description" : "Sets the identifier of the Event Hub partition that the {link EventData events} will be sent to. If the identifier is not specified, the Event Hubs service will be responsible for routing events that are sent to an available partition.",
+          "type" : "string"
+        },
+        "partition-key" : {
+          "description" : "Sets a hashing key to be provided for the batch of events, which instructs the Event Hubs service to map this key to a specific partition. The selection of a partition is stable for a given partition hashing key. Should any other batches of events be sent using the same exact partition hashing key, the Event Hubs service will route them all to the same partition. This should be specified only when there is a need to group events by partition, but there is flexi [...]
+          "type" : "string"
+        },
+        "prefetch-count" : {
+          "description" : "Sets the count used by the receiver to control the number of events the Event Hub consumer will actively receive and queue locally without regard to whether a receive operation is currently active.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "producer-async-client" : {
+          "description" : "Sets the EventHubProducerAsyncClient.An asynchronous producer responsible for transmitting EventData to a specific Event Hub, grouped together in batches. Depending on the {link CreateBatchOptions options} specified when creating an {linkEventDataBatch}, the events may be automatically routed to an available partition or specific to a partition. Use by this component to produce the data in camel producer.",
+          "type" : "string"
+        },
+        "shared-access-key" : {
+          "description" : "The generated value for the SharedAccessName",
+          "type" : "string"
+        },
+        "shared-access-name" : {
+          "description" : "The name you chose for your EventHubs SAS keys",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "azure-queue" : {
+      "type" : "object",
+      "required" : [ "containerAndQueueUri" ],
+      "properties" : {
+        "container-and-queue-uri" : {
+          "description" : "Container Queue compact Uri",
+          "type" : "string"
+        },
+        "azure-queue-client" : {
+          "description" : "The queue service client",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "credentials" : {
+          "description" : "Set the storage credentials, required in most cases",
+          "type" : "string"
+        },
+        "credentials-account-key" : {
+          "description" : "Set the storage account key used during authentication phase",
+          "type" : "string"
+        },
+        "credentials-account-name" : {
+          "description" : "Set the storage account name used during authentication phase",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "message-time-to-live" : {
+          "description" : "Message Time To Live in seconds",
+          "type" : "integer"
+        },
+        "message-visibility-delay" : {
+          "description" : "Message Visibility Delay in seconds",
+          "type" : "integer"
+        },
+        "operation" : {
+          "description" : "Queue service operation hint to the producer",
+          "default" : "listQueues",
+          "enum" : [ "listQueues", "createQueue", "deleteQueue", "addMessage", "retrieveMessage", "peekMessage", "updateMessage", "deleteMessage" ],
+          "type" : "string"
+        },
+        "queue-prefix" : {
+          "description" : "Set a prefix which can be used for listing the queues",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "validate-client-uri" : {
+          "description" : "Whether to validate the Azure client URI",
+          "default" : "true",
+          "type" : "boolean"
+        }
+      }
+    },
+    "azure-storage-blob" : {
+      "type" : "object",
+      "properties" : {
+        "account-name" : {
+          "description" : "Azure account name to be used for authentication with azure blob services",
+          "type" : "string"
+        },
+        "container-name" : {
+          "description" : "The blob container name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Access key for the associated azure account name to be used for authentication with azure blob services",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "blob-name" : {
+          "description" : "The blob name, required for consumer. However on producer, is only required for the operations on the blob level",
+          "type" : "string"
+        },
+        "blob-offset" : {
+          "description" : "Set the blob offset for the upload or download operations, default is 0",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "blob-sequence-number" : {
+          "description" : "A user-controlled value that you can use to track requests. The value of the sequence number must be between 0 and 263 - 1.The default value is 0.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "blob-service-client" : {
+          "description" : "Client to a storage account. This client does not hold any state about a particular storage account but is instead a convenient way of sending off appropriate requests to the resource on the service. It may also be used to construct URLs to blobs and containers. This client contains operations on a service account. Operations on a container are available on BlobContainerClient through getBlobContainerClient(String), and operations on a blob are available on Blo [...]
+          "type" : "string"
+        },
+        "blob-type" : {
+          "description" : "The blob type in order to initiate the appropriate settings for each blob type",
+          "default" : "blockblob",
+          "enum" : [ "blockblob", "appendblob", "pageblob" ],
+          "type" : "string"
+        },
+        "block-list-type" : {
+          "description" : "Specifies which type of blocks to return.",
+          "default" : "COMMITTED",
+          "enum" : [ "committed", "uncommitted", "all" ],
+          "type" : "string"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "close-stream-after-read" : {
+          "description" : "Close the stream after read or keep it open, default is true",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "close-stream-after-write" : {
+          "description" : "Close the stream after write or keep it open, default is true",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "commit-block-list-later" : {
+          "description" : "When is set to true, the staged blocks will not be committed directly.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "create-append-blob" : {
+          "description" : "When is set to true, the append blocks will be created when committing append blocks.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "create-page-blob" : {
+          "description" : "When is set to true, the page blob will be created when uploading page blob.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "credentials" : {
+          "description" : "StorageSharedKeyCredential can be injected to create the azure client, this holds the important authentication information",
+          "type" : "string"
+        },
+        "data-count" : {
+          "description" : "How many bytes to include in the range. Must be greater than or equal to 0 if specified.",
+          "type" : "integer"
+        },
+        "download-link-expiration" : {
+          "description" : "Override the default expiration (millis) of URL download link.",
+          "type" : "integer"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "file-dir" : {
+          "description" : "The file directory where the downloaded blobs will be saved to, this can be used in both, producer and consumer",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "max-results-per-page" : {
+          "description" : "Specifies the maximum number of blobs to return, including all BlobPrefix elements. If the request does not specify maxResultsPerPage or specifies a value greater than 5,000, the server will return up to 5,000 items.",
+          "type" : "integer"
+        },
+        "max-retry-requests" : {
+          "description" : "Specifies the maximum number of additional HTTP Get requests that will be made while reading the data from a response body.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "operation" : {
+          "description" : "The blob operation that can be used with this component on the producer",
+          "default" : "listBlobContainers",
+          "enum" : [ "listBlobContainers", "createBlobContainer", "deleteBlobContainer", "listBlobs", "getBlob", "deleteBlob", "downloadBlobToFile", "downloadLink", "uploadBlockBlob", "stageBlockBlobList", "commitBlobBlockList", "getBlobBlockList", "createAppendBlob", "commitAppendBlob", "createPageBlob", "uploadPageBlob", "resizePageBlob", "clearPageBlob", "getPageBlobRanges" ],
+          "type" : "string"
+        },
+        "page-blob-size" : {
+          "description" : "Specifies the maximum size for the page blob, up to 8 TB. The page blob size must be aligned to a 512-byte boundary.",
+          "default" : "512",
+          "type" : "integer"
+        },
+        "prefix" : {
+          "description" : "Filters the results to return only blobs whose names begin with the specified prefix. May be null to return all blobs.",
+          "type" : "string"
+        },
+        "service-client" : {
+          "description" : "Client to a storage account. This client does not hold any state about a particular storage account but is instead a convenient way of sending off appropriate requests to the resource on the service. It may also be used to construct URLs to blobs and containers. This client contains operations on a service account. Operations on a container are available on BlobContainerClient through getBlobContainerClient(String), and operations on a blob are available on Blo [...]
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "timeout" : {
+          "description" : "An optional timeout value beyond which a RuntimeException will be raised.",
+          "type" : "string"
+        }
+      }
+    },
+    "azure-storage-queue" : {
+      "type" : "object",
+      "properties" : {
+        "account-name" : {
+          "description" : "Azure account name to be used for authentication with azure queue services",
+          "type" : "string"
+        },
+        "queue-name" : {
+          "description" : "The queue resource name",
+          "type" : "string"
+        },
+        "access-key" : {
+          "description" : "Access key for the associated azure account name to be used for authentication with azure queue services",
+          "type" : "string"
+        },
+        "auto-discover-client" : {
+          "description" : "Setting the autoDiscoverClient mechanism, if true, the component will look for a client instance in the registry automatically otherwise it will skip that checking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "create-queue" : {
+          "description" : "When is set to true, the queue will be automatically created when sending messages to the queue.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "credentials" : {
+          "description" : "StorageSharedKeyCredential can be injected to create the azure client, this holds the important authentication information",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "max-messages" : {
+          "description" : "Maximum number of messages to get, if there are less messages exist in the queue than requested all the messages will be returned. If left empty only 1 message will be retrieved, the allowed range is 1 to 32 messages.",
+          "default" : "1",
+          "type" : "integer"
+        },
+        "message-id" : {
+          "description" : "The ID of the message to be deleted or updated.",
+          "type" : "string"
+        },
+        "operation" : {
+          "description" : "Queue service operation hint to the producer",
+          "enum" : [ "listQueues", "createQueue", "deleteQueue", "clearQueue", "sendMessage", "deleteMessage", "receiveMessages", "peekMessages", "updateMessage" ],
+          "type" : "string"
+        },
+        "pop-receipt" : {
+          "description" : "Unique identifier that must match for the message to be deleted or updated.",
+          "type" : "string"
+        },
+        "service-client" : {
+          "description" : "Service client to a storage account to interact with the queue service. This client does not hold any state about a particular storage account but is instead a convenient way of sending off appropriate requests to the resource on the service. This client contains all the operations for interacting with a queue account in Azure Storage. Operations allowed by the client are creating, listing, and deleting queues, retrieving and updating properties of the account, [...]
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-to-live" : {
+          "description" : "How long the message will stay alive in the queue. If unset the value will default to 7 days, if -1 is passed the message will not expire. The time to live must be -1 or any positive number. The format should be in this form: PnDTnHnMn.nS., e.g: PT20.345S -- parses as 20.345 seconds, P2D -- parses as 2 days However, in case you are using EndpointDsl/ComponentDsl, you can do something like Duration.ofSeconds() since these Java APIs are typesafe.",
+          "type" : "string"
+        },
+        "timeout" : {
+          "description" : "An optional timeout applied to the operation. If a response is not returned before the timeout concludes a RuntimeException will be thrown.",
+          "type" : "string"
+        },
+        "visibility-timeout" : {
+          "description" : "The timeout period for how long the message is invisible in the queue. The timeout must be between 1 seconds and 7 days. The format should be in this form: PnDTnHnMn.nS., e.g: PT20.345S -- parses as 20.345 seconds, P2D -- parses as 2 days However, in case you are using EndpointDsl/ComponentDsl, you can do something like Duration.ofSeconds() since these Java APIs are typesafe.",
+          "type" : "string"
+        }
+      }
+    },
+    "bean" : {
+      "type" : "object",
+      "required" : [ "beanName" ],
+      "properties" : {
+        "bean-name" : {
+          "description" : "Sets the name of the bean to invoke",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "cache" : {
+          "description" : "Use scope option instead.",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "method" : {
+          "description" : "Sets the name of the method to invoke on the bean",
+          "type" : "string"
+        },
+        "parameters" : {
+          "description" : "Used for configuring additional properties on the bean",
+          "type" : "string"
+        },
+        "scope" : {
+          "description" : "Scope of bean. When using singleton scope (default) the bean is created or looked up only once and reused for the lifetime of the endpoint. The bean should be thread-safe in case concurrent threads is calling the bean at the same time. When using request scope the bean is created or looked up once per request (exchange). This can be used if you want to store state on a bean while processing a request and you want to call the same bean instance multiple times wh [...]
+          "default" : "Singleton",
+          "enum" : [ "Singleton", "Request", "Prototype" ],
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "bean-validator" : {
+      "type" : "object",
+      "required" : [ "label" ],
+      "properties" : {
+        "label" : {
+          "description" : "Where label is an arbitrary text value describing the endpoint",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "constraint-validator-factory" : {
+          "description" : "To use a custom ConstraintValidatorFactory",
+          "type" : "string"
+        },
+        "group" : {
+          "description" : "To use a custom validation group",
+          "default" : "javax.validation.groups.Default",
+          "type" : "string"
+        },
+        "ignore-xml-configuration" : {
+          "description" : "Whether to ignore data from the META-INF/validation.xml file.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "message-interpolator" : {
+          "description" : "To use a custom MessageInterpolator",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "traversable-resolver" : {
+          "description" : "To use a custom TraversableResolver",
+          "type" : "string"
+        },
+        "validation-provider-resolver" : {
+          "description" : "To use a a custom ValidationProviderResolver",
+          "type" : "string"
+        },
+        "validator-factory" : {
+          "description" : "To use a custom ValidatorFactory",
+          "type" : "string"
+        }
+      }
+    },
+    "beanstalk" : {
+      "type" : "object",
+      "properties" : {
+        "connection-settings" : {
+          "description" : "Connection settings host:port/tube",
+          "type" : "string"
+        },
+        "await-job" : {
+          "description" : "Whether to wait for job to complete before ack the job from beanstalk",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "command" : {
+          "description" : "put means to put the job into Beanstalk. Job body is specified in the Camel message body. Job ID will be returned in beanstalk.jobId message header. delete, release, touch or bury expect Job ID in the message header beanstalk.jobId. Result of the operation is returned in beanstalk.result message header kick expects the number of jobs to kick in the message body and returns the number of jobs actually kicked out in the message header beanstalk.result.",
+          "enum" : [ "bury", "release", "put", "touch", "delete", "kick" ],
+          "type" : "string"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "job-delay" : {
+          "description" : "Job delay in seconds.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "job-priority" : {
+          "description" : "Job priority. (0 is the highest, see Beanstalk protocol)",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "job-time-to-run" : {
+          "description" : "Job time to run in seconds. (when 0, the beanstalkd daemon raises it to 1 automatically, see Beanstalk protocol)",
+          "default" : "60",
+          "type" : "integer"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "on-failure" : {
+          "description" : "Command to use when processing failed.",
+          "enum" : [ "bury", "release", "put", "touch", "delete", "kick" ],
+          "type" : "string"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "use-block-io" : {
+          "description" : "Whether to use blockIO.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        }
+      }
+    },
+    "bonita" : {
+      "type" : "object",
+      "required" : [ "operation" ],
+      "properties" : {
+        "operation" : {
+          "description" : "Operation to use",
+          "enum" : [ "startCase" ],
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "hostname" : {
+          "description" : "Hostname where Bonita engine runs",
+          "default" : "localhost",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "password" : {
+          "description" : "Password to authenticate to Bonita engine.",
+          "type" : "string"
+        },
+        "port" : {
+          "description" : "Port of the server hosting Bonita engine",
+          "default" : "8080",
+          "type" : "string"
+        },
+        "process-name" : {
+          "description" : "Name of the process involved in the operation",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "username" : {
+          "description" : "Username to authenticate to Bonita engine.",
+          "type" : "string"
+        }
+      }
+    },
+    "box" : {
+      "type" : "object",
+      "required" : [ "apiName", "methodName" ],
+      "properties" : {
+        "api-name" : {
+          "description" : "What kind of operation to perform",
+          "enum" : [ "COLLABORATIONS", "COMMENTS", "EVENT_LOGS", "FILES", "FOLDERS", "GROUPS", "EVENTS", "SEARCH", "TASKS", "USERS" ],
+          "type" : "string"
+        },
+        "method-name" : {
+          "description" : "What sub operation to use for the selected operation",
+          "type" : "string"
+        },
+        "access-token-cache" : {
+          "description" : "Custom Access Token Cache for storing and retrieving access tokens.",
+          "type" : "string"
+        },
+        "authentication-type" : {
+          "description" : "The type of authentication for connection. Types of Authentication: STANDARD_AUTHENTICATION - OAuth 2.0 (3-legged) SERVER_AUTHENTICATION - OAuth 2.0 with JSON Web Tokens",
+          "default" : "APP_USER_AUTHENTICATION",
+          "type" : "string"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "client-id" : {
+          "description" : "Box application client ID",
+          "type" : "string"
+        },
+        "client-secret" : {
+          "description" : "Box application client secret",
+          "type" : "string"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "encryption-algorithm" : {
+          "description" : "The type of encryption algorithm for JWT. Supported Algorithms: RSA_SHA_256 RSA_SHA_384 RSA_SHA_512",
+          "default" : "RSA_SHA_256",
+          "enum" : [ "RSA_SHA_256", "RSA_SHA_384", "RSA_SHA_512" ],
+          "type" : "string"
+        },
+        "enterprise-id" : {
+          "description" : "The enterprise ID to use for an App Enterprise.",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "http-params" : {
+          "description" : "Custom HTTP params for settings like proxy host",
+          "type" : "string"
+        },
+        "in-body" : {
+          "description" : "Sets the name of a parameter to be passed in the exchange In Body",
+          "type" : "string"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "max-cache-entries" : {
+          "description" : "The maximum number of access tokens in cache.",
+          "default" : "100",
+          "type" : "integer"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "private-key-file" : {
+          "description" : "The private key for generating the JWT signature.",
+          "type" : "string"
+        },
+        "private-key-password" : {
+          "description" : "The password for the private key.",
+          "type" : "string"
+        },
+        "public-key-id" : {
+          "description" : "The ID for public key for validating the JWT signature.",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "ssl-context-parameters" : {
+          "description" : "To configure security using SSLContextParameters.",
+          "type" : "string"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "user-id" : {
+          "description" : "The user ID to use for an App User.",
+          "type" : "string"
+        },
+        "user-name" : {
+          "description" : "Box user name, MUST be provided",
+          "type" : "string"
+        },
+        "user-password" : {
+          "description" : "Box user password, MUST be provided if authSecureStorage is not set, or returns null on first call",
+          "type" : "string"
+        }
+      }
+    },
+    "braintree" : {
+      "type" : "object",
+      "required" : [ "apiName" ],
+      "properties" : {
+        "api-name" : {
+          "description" : "What kind of operation to perform",
+          "enum" : [ "ADDON", "ADDRESS", "CLIENTTOKEN", "CREDITCARDVERIFICATION", "CUSTOMER", "DISCOUNT", "DISPUTE", "DOCUMENTUPLOAD", "MERCHANTACCOUNT", "PAYMENTMETHOD", "PAYMENTMETHODNONCE", "PLAN", "REPORT", "SETTLEMENTBATCHSUMMARY", "SUBSCRIPTION", "TRANSACTION", "WEBHOOKNOTIFICATION" ],
+          "type" : "string"
+        },
+        "method-name" : {
+          "description" : "What sub operation to use for the selected operation",
+          "type" : "string"
+        },
+        "access-token" : {
+          "description" : "The access token granted by a merchant to another in order to process transactions on their behalf. Used in place of environment, merchant id, public key and private key fields.",
+          "type" : "string"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "environment" : {
+          "description" : "The environment Either SANDBOX or PRODUCTION",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "http-log-level" : {
+          "description" : "Set logging level for http calls, see java.util.logging.Level",
+          "enum" : [ "OFF", "SEVERE", "WARNING", "INFO", "CONFIG", "FINE", "FINER", "FINEST", "ALL" ],
+          "type" : "string"
+        },
+        "http-log-name" : {
+          "description" : "Set log category to use to log http calls.",
+          "default" : "Braintree",
+          "type" : "string"
+        },
+        "http-read-timeout" : {
+          "description" : "Set read timeout for http calls.",
+          "type" : "integer"
+        },
+        "in-body" : {
+          "description" : "Sets the name of a parameter to be passed in the exchange In Body",
+          "type" : "string"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "log-handler-enabled" : {
+          "description" : "Sets whether to enable the BraintreeLogHandler. It may be desirable to set this to 'false' where an existing JUL - SLF4J logger bridge is on the classpath. This option can also be configured globally on the BraintreeComponent.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "merchant-id" : {
+          "description" : "The merchant id provided by Braintree.",
+          "type" : "string"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "private-key" : {
+          "description" : "The private key provided by Braintree.",
+          "type" : "string"
+        },
+        "proxy-host" : {
+          "description" : "The proxy host",
+          "type" : "string"
+        },
+        "proxy-port" : {
+          "description" : "The proxy port",
+          "type" : "integer"
+        },
+        "public-key" : {
+          "description" : "The public key provided by Braintree.",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        }
+      }
+    },
+    "browse" : {
+      "type" : "object",
+      "required" : [ "name" ],
+      "properties" : {
+        "name" : {
+          "description" : "A name which can be any string to uniquely identify the endpoint",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "caffeine-cache" : {
+      "type" : "object",
+      "required" : [ "cacheName" ],
+      "properties" : {
+        "cache-name" : {
+          "description" : "the cache name",
+          "type" : "string"
+        },
+        "action" : {
+          "description" : "To configure the default cache action. If an action is set in the message header, then the operation from the header takes precedence.",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "cache" : {
+          "description" : "To configure an already instantiated cache to be used",
+          "type" : "string"
+        },
+        "cache-loader" : {
+          "description" : "To configure a CacheLoader in case of a LoadCache use",
+          "type" : "string"
+        },
+        "create-cache-if-not-exist" : {
+          "description" : "Configure if a cache need to be created if it does exist or can't be pre-configured.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "eviction-type" : {
+          "description" : "Set the eviction Type for this cache",
+          "default" : "SIZE_BASED",
+          "enum" : [ "size_based", "time_based" ],
+          "type" : "string"
+        },
+        "expire-after-access-time" : {
+          "description" : "Set the expire After Access Time in case of time based Eviction (in seconds)",
+          "default" : "300",
+          "type" : "integer"
+        },
+        "expire-after-write-time" : {
+          "description" : "Set the expire After Access Write in case of time based Eviction (in seconds)",
+          "default" : "300",
+          "type" : "integer"
+        },
+        "initial-capacity" : {
+          "description" : "Set the initial Capacity for the cache",
+          "default" : "10000",
+          "type" : "integer"
+        },
+        "key" : {
+          "description" : "To configure the default action key. If a key is set in the message header, then the key from the header takes precedence.",
+          "type" : "string"
+        },
+        "key-type" : {
+          "description" : "The cache key type, default java.lang.Object",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "maximum-size" : {
+          "description" : "Set the maximum size for the cache",
+          "default" : "10000",
+          "type" : "integer"
+        },
+        "removal-listener" : {
+          "description" : "Set a specific removal Listener for the cache",
+          "type" : "string"
+        },
+        "stats-counter" : {
+          "description" : "Set a specific Stats Counter for the cache stats",
+          "type" : "string"
+        },
+        "stats-enabled" : {
+          "description" : "To enable stats on the cache",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "value-type" : {
+          "description" : "The cache value type, default java.lang.Object",
+          "type" : "string"
+        }
+      }
+    },
+    "caffeine-loadcache" : {
+      "type" : "object",
+      "required" : [ "cacheName" ],
+      "properties" : {
+        "cache-name" : {
+          "description" : "the cache name",
+          "type" : "string"
+        },
+        "action" : {
+          "description" : "To configure the default cache action. If an action is set in the message header, then the operation from the header takes precedence.",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "cache" : {
+          "description" : "To configure an already instantiated cache to be used",
+          "type" : "string"
+        },
+        "cache-loader" : {
+          "description" : "To configure a CacheLoader in case of a LoadCache use",
+          "type" : "string"
+        },
+        "create-cache-if-not-exist" : {
+          "description" : "Configure if a cache need to be created if it does exist or can't be pre-configured.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "eviction-type" : {
+          "description" : "Set the eviction Type for this cache",
+          "default" : "SIZE_BASED",
+          "enum" : [ "size_based", "time_based" ],
+          "type" : "string"
+        },
+        "expire-after-access-time" : {
+          "description" : "Set the expire After Access Time in case of time based Eviction (in seconds)",
+          "default" : "300",
+          "type" : "integer"
+        },
+        "expire-after-write-time" : {
+          "description" : "Set the expire After Access Write in case of time based Eviction (in seconds)",
+          "default" : "300",
+          "type" : "integer"
+        },
+        "initial-capacity" : {
+          "description" : "Set the initial Capacity for the cache",
+          "default" : "10000",
+          "type" : "integer"
+        },
+        "key" : {
+          "description" : "To configure the default action key. If a key is set in the message header, then the key from the header takes precedence.",
+          "type" : "string"
+        },
+        "key-type" : {
+          "description" : "The cache key type, default java.lang.Object",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "maximum-size" : {
+          "description" : "Set the maximum size for the cache",
+          "default" : "10000",
+          "type" : "integer"
+        },
+        "removal-listener" : {
+          "description" : "Set a specific removal Listener for the cache",
+          "type" : "string"
+        },
+        "stats-counter" : {
+          "description" : "Set a specific Stats Counter for the cache stats",
+          "type" : "string"
+        },
+        "stats-enabled" : {
+          "description" : "To enable stats on the cache",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "value-type" : {
+          "description" : "The cache value type, default java.lang.Object",
+          "type" : "string"
+        }
+      }
+    },
+    "chatscript" : {
+      "type" : "object",
+      "required" : [ "botName", "host" ],
+      "properties" : {
+        "bot-name" : {
+          "description" : "Name of the Bot in CS to converse with",
+          "type" : "string"
+        },
+        "host" : {
+          "description" : "Hostname or IP of the server on which CS server is running",
+          "type" : "string"
+        },
+        "port" : {
+          "description" : "Port on which ChatScript is listening to",
+          "default" : "1024",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "chat-user-name" : {
+          "description" : "Username who initializes the CS conversation. To be set when chat is initialized from camel route",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "reset-chat" : {
+          "description" : "Issues :reset command to start a new conversation everytime",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "chunk" : {
+      "type" : "object",
+      "required" : [ "resourceUri" ],
+      "properties" : {
+        "resource-uri" : {
+          "description" : "Path to the resource. You can prefix with: classpath, file, http, ref, or bean. classpath, file and http loads the resource using these protocols (classpath is default). ref will lookup the resource in the registry. bean will call a method on a bean to be used as the resource. For bean you can specify the method name after dot, eg bean:myBean.myMethod.",
+          "type" : "string"
+        },
+        "allow-context-map-all" : {
+          "description" : "Sets whether the context map should allow access to all details. By default only the message body and headers can be accessed. This option can be enabled for full access to the current Exchange and CamelContext. Doing so impose a potential security risk as this opens access to the full power of CamelContext API.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "allow-template-from-header" : {
+          "description" : "Whether to allow to use resource template from header or not (default false). Enabling this allows to specify dynamic templates via message header. However this can be seen as a potential security vulnerability if the header is coming from a malicious user, so use this with care.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "content-cache" : {
+          "description" : "Sets whether to use resource content cache or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "encoding" : {
+          "description" : "Define the encoding of the body",
+          "type" : "string"
+        },
+        "extension" : {
+          "description" : "Define the file extension of the template",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "theme-folder" : {
+          "description" : "Define the themes folder to scan",
+          "type" : "string"
+        },
+        "theme-layer" : {
+          "description" : "Define the theme layer to elaborate",
+          "type" : "string"
+        },
+        "theme-subfolder" : {
+          "description" : "Define the themes subfolder to scan",
+          "type" : "string"
+        }
+      }
+    },
+    "class" : {
+      "type" : "object",
+      "required" : [ "beanName" ],
+      "properties" : {
+        "bean-name" : {
+          "description" : "Sets the name of the bean to invoke",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "cache" : {
+          "description" : "Use scope option instead.",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "method" : {
+          "description" : "Sets the name of the method to invoke on the bean",
+          "type" : "string"
+        },
+        "parameters" : {
+          "description" : "Used for configuring additional properties on the bean",
+          "type" : "string"
+        },
+        "scope" : {
+          "description" : "Scope of bean. When using singleton scope (default) the bean is created or looked up only once and reused for the lifetime of the endpoint. The bean should be thread-safe in case concurrent threads is calling the bean at the same time. When using request scope the bean is created or looked up once per request (exchange). This can be used if you want to store state on a bean while processing a request and you want to call the same bean instance multiple times wh [...]
+          "default" : "Singleton",
+          "enum" : [ "Singleton", "Request", "Prototype" ],
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "cm-sms" : {
+      "type" : "object",
+      "required" : [ "host", "defaultFrom", "productToken" ],
+      "properties" : {
+        "host" : {
+          "description" : "SMS Provider HOST with scheme",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "default-from" : {
+          "description" : "This is the sender name. The maximum length is 11 characters.",
+          "type" : "string"
+        },
+        "default-max-number-of-parts" : {
+          "description" : "If it is a multipart message forces the max number. Message can be truncated. Technically the gateway will first check if a message is larger than 160 characters, if so, the message will be cut into multiple 153 characters parts limited by these parameters.",
+          "default" : "8",
+          "type" : "integer"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "product-token" : {
+          "description" : "The unique token to use",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "test-connection-on-startup" : {
+          "description" : "Whether to test the connection to the SMS Gateway on startup",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "cmis" : {
+      "type" : "object",
+      "required" : [ "cmsUrl" ],
+      "properties" : {
+        "cms-url" : {
+          "description" : "URL to the cmis repository",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "page-size" : {
+          "description" : "Number of nodes to retrieve per page",
+          "default" : "100",
+          "type" : "integer"
+        },
+        "password" : {
+          "description" : "Password for the cmis repository",
+          "type" : "string"
+        },
+        "query" : {
+          "description" : "The cmis query to execute against the repository. If not specified, the consumer will retrieve every node from the content repository by iterating the content tree recursively",
+          "type" : "string"
+        },
+        "query-mode" : {
+          "description" : "If true, will execute the cmis query from the message body and return result, otherwise will create a node in the cmis repository",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "read-content" : {
+          "description" : "If set to true, the content of document node will be retrieved in addition to the properties",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "read-count" : {
+          "description" : "Max number of nodes to read",
+          "type" : "integer"
+        },
+        "repository-id" : {
+          "description" : "The Id of the repository to use. If not specified the first available repository is used",
+          "type" : "string"
+        },
+        "session-facade-factory" : {
+          "description" : "To use a custom CMISSessionFacadeFactory to create the CMISSessionFacade instances",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "username" : {
+          "description" : "Username for the cmis repository",
+          "type" : "string"
+        }
+      }
+    },
+    "coap" : {
+      "type" : "object",
+      "properties" : {
+        "uri" : {
+          "description" : "The URI for the CoAP endpoint",
+          "type" : "string"
+        },
+        "alias" : {
+          "description" : "Sets the alias used to query the KeyStore for the private key and certificate. This parameter is used when we are enabling TLS with certificates on the service side, and similarly on the client side when TLS is used with certificates and client authentication. If the parameter is not specified then the default behavior is to use the first alias in the keystore that contains a key entry. This configuration parameter does not apply to configuring TLS via a Raw Pu [...]
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "cipher-suites" : {
+          "description" : "Sets the cipherSuites String. This is a comma separated String of ciphersuites to configure. If it is not specified, then it falls back to getting the ciphersuites from the sslContextParameters object.",
+          "type" : "string"
+        },
+        "client-authentication" : {
+          "description" : "Sets the configuration options for server-side client-authentication requirements. The value must be one of NONE, WANT, REQUIRE. If this value is not specified, then it falls back to checking the sslContextParameters.getServerParameters().getClientAuthentication() value.",
+          "type" : "string"
+        },
+        "coap-method-restrict" : {
+          "description" : "Comma separated list of methods that the CoAP consumer will bind to. The default is to bind to all methods (DELETE, GET, POST, PUT).",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "private-key" : {
+          "description" : "Set the configured private key for use with Raw Public Key.",
+          "type" : "string"
+        },
+        "psk-store" : {
+          "description" : "Set the PskStore to use for pre-shared key.",
+          "type" : "string"
+        },
+        "public-key" : {
+          "description" : "Set the configured public key for use with Raw Public Key.",
+          "type" : "string"
+        },
+        "recommended-cipher-suites-only" : {
+          "description" : "The CBC cipher suites are not recommended. If you want to use them, you first need to set the recommendedCipherSuitesOnly option to false.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "ssl-context-parameters" : {
+          "description" : "Set the SSLContextParameters object for setting up TLS. This is required for coapstcp, and for coaps when we are using certificates for TLS (as opposed to RPK or PKS).",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "trusted-rpk-store" : {
+          "description" : "Set the TrustedRpkStore to use to determine trust in raw public keys.",
+          "type" : "string"
+        }
+      }
+    },
+    "coaps" : {
+      "type" : "object",
+      "$ref" : "#/definitions/coap"
+    },
+    "coap+tcp" : {
+      "type" : "object",
+      "$ref" : "#/definitions/coap"
+    },
+    "coaps+tcp" : {
+      "type" : "object",
+      "$ref" : "#/definitions/coap"
+    },
+    "cometd" : {
+      "type" : "object",
+      "required" : [ "channelName", "host", "port" ],
+      "properties" : {
+        "channel-name" : {
+          "description" : "The channelName represents a topic that can be subscribed to by the Camel endpoints.",
+          "type" : "string"
+        },
+        "host" : {
+          "description" : "Hostname",
+          "type" : "string"
+        },
+        "port" : {
+          "description" : "Host port number",
+          "type" : "integer"
+        },
+        "allowed-origins" : {
+          "description" : "The origins domain that support to cross, if the crosssOriginFilterOn is true",
+          "default" : "*",
+          "type" : "string"
+        },
+        "base-resource" : {
+          "description" : "The root directory for the web resources or classpath. Use the protocol file: or classpath: depending if you want that the component loads the resource from file system or classpath. Classpath is required for OSGI deployment where the resources are packaged in the jar",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "cross-origin-filter-on" : {
+          "description" : "If true, the server will support for cross-domain filtering",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "disconnect-local-session" : {
+          "description" : "Whether to disconnect local sessions after publishing a message to its channel. Disconnecting local session is needed as they are not swept by default by CometD, and therefore you can run out of memory.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "filter-path" : {
+          "description" : "The filterPath will be used by the CrossOriginFilter, if the crosssOriginFilterOn is true",
+          "type" : "string"
+        },
+        "interval" : {
+          "description" : "The client side poll timeout in milliseconds. How long a client will wait between reconnects",
+          "type" : "integer"
+        },
+        "json-commented" : {
+          "description" : "If true, the server will accept JSON wrapped in a comment and will generate JSON wrapped in a comment. This is a defence against Ajax Hijacking.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "log-level" : {
+          "description" : "Logging level. 0=none, 1=info, 2=debug.",
+          "default" : "1",
+          "enum" : [ "0", "1", "2" ],
+          "type" : "integer"
+        },
+        "max-interval" : {
+          "description" : "The max client side poll timeout in milliseconds. A client will be removed if a connection is not received in this time.",
+          "default" : "30000",
+          "type" : "integer"
+        },
+        "multi-frame-interval" : {
+          "description" : "The client side poll timeout, if multiple connections are detected from the same browser.",
+          "default" : "1500",
+          "type" : "integer"
+        },
+        "session-headers-enabled" : {
+          "description" : "Whether to include the server session headers in the Camel message when creating a Camel Message for incoming requests.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "timeout" : {
+          "description" : "The server side poll timeout in milliseconds. This is how long the server will hold a reconnect request before responding.",
+          "default" : "240000",
+          "type" : "integer"
+        }
+      }
+    },
+    "cometds" : {
+      "type" : "object",
+      "$ref" : "#/definitions/cometd"
+    },
+    "consul" : {
+      "type" : "object",
+      "required" : [ "apiEndpoint" ],
+      "properties" : {
+        "api-endpoint" : {
+          "description" : "The API endpoint",
+          "type" : "string"
+        },
+        "acl-token" : {
+          "description" : "Sets the ACL token to be used with Consul",
+          "type" : "string"
+        },
+        "action" : {
+          "description" : "The default action. Can be overridden by CamelConsulAction",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "block-seconds" : {
+          "description" : "The second to wait for a watch event, default 10 seconds",
+          "default" : "10",
+          "type" : "integer"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "connect-timeout" : {
+          "description" : "Connect timeout for OkHttpClient",
+          "type" : "string"
+        },
+        "connect-timeout-millis" : {
+          "description" : "Connect timeout for OkHttpClient. Deprecation note: Use connectTimeout instead",
+          "type" : "integer"
+        },
+        "consistency-mode" : {
+          "description" : "The consistencyMode used for queries, default ConsistencyMode.DEFAULT",
+          "default" : "DEFAULT",
+          "enum" : [ "DEFAULT", "STALE", "CONSISTENT" ],
+          "type" : "string"
+        },
+        "consul-client" : {
+          "description" : "Reference to a com.orbitz.consul.Consul in the registry.",
+          "type" : "string"
+        },
+        "datacenter" : {
+          "description" : "The data center",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "first-index" : {
+          "description" : "The first index for watch for, default 0",
+          "default" : "0",
+          "type" : "string"
+        },
+        "key" : {
+          "description" : "The default key. Can be overridden by CamelConsulKey",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "near-node" : {
+          "description" : "The near node to use for queries.",
+          "type" : "string"
+        },
+        "node-meta" : {
+          "description" : "The note meta-data to use for queries.",
+          "type" : "string"
+        },
+        "password" : {
+          "description" : "Sets the password to be used for basic authentication",
+          "type" : "string"
+        },
+        "ping-instance" : {
+          "description" : "Configure if the AgentClient should attempt a ping before returning the Consul instance",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "read-timeout" : {
+          "description" : "Read timeout for OkHttpClient",
+          "type" : "string"
+        },
+        "read-timeout-millis" : {
+          "description" : "Read timeout for OkHttpClient. Deprecation note: Use readTimeout instead",
+          "type" : "integer"
+        },
+        "recursive" : {
+          "description" : "Recursively watch, default false",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "ssl-context-parameters" : {
+          "description" : "SSL configuration using an org.apache.camel.support.jsse.SSLContextParameters instance.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "tags" : {
+          "description" : "Set tags. You can separate multiple tags by comma.",
+          "type" : "string"
+        },
+        "url" : {
+          "description" : "The Consul agent URL",
+          "type" : "string"
+        },
+        "user-name" : {
+          "description" : "Sets the username to be used for basic authentication",
+          "type" : "string"
+        },
+        "value-as-string" : {
+          "description" : "Default to transform values retrieved from Consul i.e. on KV endpoint to string.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "write-timeout" : {
+          "description" : "Write timeout for OkHttpClient",
+          "type" : "string"
+        },
+        "write-timeout-millis" : {
+          "description" : "Write timeout for OkHttpClient. Deprecation note: Use writeTimeout instead",
+          "type" : "string"
+        }
+      }
+    },
+    "controlbus" : {
+      "type" : "object",
+      "required" : [ "command" ],
+      "properties" : {
+        "command" : {
+          "description" : "Command can be either route or language",
+          "enum" : [ "route", "language" ],
+          "type" : "string"
+        },
+        "language" : {
+          "description" : "Allows you to specify the name of a Language to use for evaluating the message body. If there is any result from the evaluation, then the result is put in the message body.",
+          "enum" : [ "bean", "constant", "el", "exchangeProperty", "file", "groovy", "header", "jsonpath", "mvel", "ognl", "ref", "simple", "spel", "sql", "terser", "tokenize", "xpath", "xquery", "xtokenize" ],
+          "type" : "string"
+        },
+        "action" : {
+          "description" : "To denote an action that can be either: start, stop, or status. To either start or stop a route, or to get the status of the route as output in the message body. You can use suspend and resume from Camel 2.11.1 onwards to either suspend or resume a route. And from Camel 2.11.1 onwards you can use stats to get performance statics returned in XML format; the routeId option can be used to define which route to get the performance stats for, if routeId is not defin [...]
+          "enum" : [ "start", "stop", "suspend", "resume", "restart", "status", "stats" ],
+          "type" : "string"
+        },
+        "async" : {
+          "description" : "Whether to execute the control bus task asynchronously. Important: If this option is enabled, then any result from the task is not set on the Exchange. This is only possible if executing tasks synchronously.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "logging-level" : {
+          "description" : "Logging level used for logging when task is done, or if any exceptions occurred during processing the task.",
+          "default" : "INFO",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "restart-delay" : {
+          "description" : "The delay in millis to use when restarting a route.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "route-id" : {
+          "description" : "To specify a route by its id. The special keyword current indicates the current route.",
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        }
+      }
+    },
+    "corda" : {
+      "type" : "object",
+      "required" : [ "node" ],
+      "properties" : {
+        "node" : {
+          "description" : "The url for the corda node",
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "contract-state-class" : {
+          "description" : "A contract state (or just state) contains opaque data used by a contract program. It can be thought of as a disk file that the program can use to persist data across transactions. States are immutable: once created they are never updated, instead, any changes must generate a new successor state. States can be updated (consumed) only once: the notary is responsible for ensuring there is no double spending by only signing a transaction if the input states are all [...]
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "flow-logic-arguments" : {
+          "description" : "Start the given flow with the given arguments, returning an Observable with a single observation of the result of running the flow. The flowLogicClass must be annotated with net.corda.core.flows.StartableByRPC.",
+          "type" : "string"
+        },
+        "flow-logic-class" : {
+          "description" : "Start the given flow with the given arguments, returning an Observable with a single observation of the result of running the flow. The flowLogicClass must be annotated with net.corda.core.flows.StartableByRPC.",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "operation" : {
+          "description" : "Operation to use",
+          "type" : "string"
+        },
+        "page-specification" : {
+          "description" : "PageSpecification allows specification of a page number (starting from 1) and page size (defaulting to 200 with a maximum page size of (Integer.MAX_INT) Note: we default the page number to 200 to enable queries without requiring a page specification but enabling detection of large results sets that fall out of the 200 requirement. Max page size should be used with extreme caution as results may exceed your JVM memory footprint.",
+          "default" : "200",
+          "type" : "string"
+        },
+        "password" : {
+          "description" : "Password for login",
+          "type" : "string"
+        },
+        "process-snapshot" : {
+          "description" : "Whether to process snapshots or not",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "query-criteria" : {
+          "description" : "QueryCriteria assumes underlying schema tables are correctly indexed for performance.",
+          "type" : "string"
+        },
+        "sort" : {
+          "description" : "Sort allows specification of a set of entity attribute names and their associated directionality and null handling, to be applied upon processing a query specification.",
+          "enum" : [ "ASC", "DESC" ],
+          "type" : "string"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "username" : {
+          "description" : "Username for login",
+          "type" : "string"
+        }
+      }
+    },
+    "couchbase" : {
+      "type" : "object",
+      "required" : [ "hostname", "protocol" ],
+      "properties" : {
+        "hostname" : {
+          "description" : "The hostname to use",
+          "type" : "string"
+        },
+        "port" : {
+          "description" : "The port number to use",
+          "default" : "8091",
+          "type" : "integer"
+        },
+        "protocol" : {
+          "description" : "The protocol to use",
+          "type" : "string"
+        },
+        "additional-hosts" : {
+          "description" : "The additional hosts",
+          "type" : "string"
+        },
+        "auto-start-id-for-inserts" : {
+          "description" : "Define if we want an autostart Id when we are doing an insert operation",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "backoff-error-threshold" : {
+          "description" : "The number of subsequent error polls (failed due some error) that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-idle-threshold" : {
+          "description" : "The number of subsequent idle polls that should happen before the backoffMultipler should kick-in.",
+          "type" : "integer"
+        },
+        "backoff-multiplier" : {
+          "description" : "To let the scheduled polling consumer backoff if there has been a number of subsequent idles/errors in a row. The multiplier is then the number of polls that will be skipped before the next actual attempt is happening again. When this option is in use then backoffIdleThreshold and/or backoffErrorThreshold must also be configured.",
+          "type" : "integer"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bucket" : {
+          "description" : "The bucket to use",
+          "type" : "string"
+        },
+        "collection" : {
+          "description" : "The collection to use",
+          "type" : "string"
+        },
+        "consumer-processed-strategy" : {
+          "description" : "Define the consumer Processed strategy to use",
+          "default" : "none",
+          "type" : "string"
+        },
+        "delay" : {
+          "description" : "Milliseconds before the next poll.",
+          "default" : "500",
+          "type" : "integer"
+        },
+        "descending" : {
+          "description" : "Define if this operation is descending or not",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "design-document-name" : {
+          "description" : "The design document name to use",
+          "default" : "beer",
+          "type" : "string"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "greedy" : {
+          "description" : "If greedy is enabled, then the ScheduledPollConsumer will run immediately again, if the previous run polled 1 or more messages.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "initial-delay" : {
+          "description" : "Milliseconds before the first poll starts.",
+          "default" : "1000",
+          "type" : "integer"
+        },
+        "key" : {
+          "description" : "The key to use",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "limit" : {
+          "description" : "The output limit to use",
+          "default" : "-1",
+          "type" : "integer"
+        },
+        "operation" : {
+          "description" : "The operation to do",
+          "default" : "CCB_PUT",
+          "type" : "string"
+        },
+        "password" : {
+          "description" : "The password to use",
+          "type" : "string"
+        },
+        "persist-to" : {
+          "description" : "Where to persist the data",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "poll-strategy" : {
+          "description" : "A pluggable org.apache.camel.PollingConsumerPollingStrategy allowing you to provide your custom implementation to control error handling usually occurred during the poll operation before an Exchange have been created and being routed in Camel.",
+          "type" : "string"
+        },
+        "producer-retry-attempts" : {
+          "description" : "Define the number of retry attempts",
+          "default" : "2",
+          "type" : "integer"
+        },
+        "producer-retry-pause" : {
+          "description" : "Define the retry pause between different attempts",
+          "default" : "5000",
+          "type" : "integer"
+        },
+        "query-timeout" : {
+          "description" : "Define the operation timeout in milliseconds",
+          "default" : "2500",
+          "type" : "string"
+        },
+        "range-end-key" : {
+          "description" : "Define a range for the end key",
+          "type" : "string"
+        },
+        "range-start-key" : {
+          "description" : "Define a range for the start key",
+          "type" : "string"
+        },
+        "repeat-count" : {
+          "description" : "Specifies a maximum limit of number of fires. So if you set it to 1, the scheduler will only fire once. If you set it to 5, it will only fire five times. A value of zero or negative means fire forever.",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "replicate-to" : {
+          "description" : "Where to replicate the data",
+          "default" : "0",
+          "type" : "integer"
+        },
+        "run-logging-level" : {
+          "description" : "The consumer logs a start/complete log line when it polls. This option allows you to configure the logging level for that.",
+          "default" : "TRACE",
+          "enum" : [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ],
+          "type" : "string"
+        },
+        "scheduled-executor-service" : {
+          "description" : "Allows for configuring a custom/shared thread pool to use for the consumer. By default each consumer has its own single threaded thread pool.",
+          "type" : "string"
+        },
+        "scheduler" : {
+          "description" : "To use a cron scheduler from either camel-spring or camel-quartz component. Use value spring or quartz for built in scheduler",
+          "default" : "none",
+          "type" : "string"
+        },
+        "scheduler-properties" : {
+          "description" : "To configure additional properties when using a custom scheduler or any of the Quartz, Spring based scheduler.",
+          "type" : "string"
+        },
+        "scope" : {
+          "description" : "The scope to use",
+          "type" : "string"
+        },
+        "send-empty-message-when-idle" : {
+          "description" : "If the polling consumer did not poll any files, you can enable this option to send an empty message (no body) instead.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "skip" : {
+          "description" : "Define the skip to use",
+          "default" : "-1",
+          "type" : "integer"
+        },
+        "start-scheduler" : {
+          "description" : "Whether the scheduler should be auto started.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "starting-id-for-inserts-from" : {
+          "description" : "Define the starting Id where we are doing an insert operation",
+          "type" : "integer"
+        },
+        "synchronous" : {
+          "description" : "Sets whether synchronous processing should be strictly used, or Camel is allowed to use asynchronous processing (if supported).",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "time-unit" : {
+          "description" : "Time unit for initialDelay and delay options.",
+          "default" : "MILLISECONDS",
+          "enum" : [ "NANOSECONDS", "MICROSECONDS", "MILLISECONDS", "SECONDS", "MINUTES", "HOURS", "DAYS" ],
+          "type" : "string"
+        },
+        "use-fixed-delay" : {
+          "description" : "Controls if fixed delay or fixed rate is used. See ScheduledExecutorService in JDK for details.",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "username" : {
+          "description" : "The username to use",
+          "type" : "string"
+        },
+        "view-name" : {
+          "description" : "The view name to use",
+          "default" : "brewery_beers",
+          "type" : "string"
+        }
+      }
+    },
+    "couchdb" : {
+      "type" : "object",
+      "required" : [ "database", "hostname", "protocol" ],
+      "properties" : {
+        "database" : {
+          "description" : "Name of the database to use",
+          "type" : "string"
+        },
+        "hostname" : {
+          "description" : "Hostname of the running couchdb instance",
+          "type" : "string"
+        },
+        "port" : {
+          "description" : "Port number for the running couchdb instance",
+          "default" : "5984",
+          "type" : "integer"
+        },
+        "protocol" : {
+          "description" : "The protocol to use for communicating with the database.",
+          "enum" : [ "http", "https" ],
+          "type" : "string"
+        },
+        "basic-property-binding" : {
+          "description" : "Whether the endpoint should use basic property binding (Camel 2.x) or the newer property binding with additional capabilities",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "bridge-error-handler" : {
+          "description" : "Allows for bridging the consumer to the Camel routing Error Handler, which mean any exceptions occurred while the consumer is trying to pickup incoming messages, or the likes, will now be processed as a message and handled by the routing Error Handler. By default the consumer will use the org.apache.camel.spi.ExceptionHandler to deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "create-database" : {
+          "description" : "Creates the database if it does not already exist",
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "deletes" : {
+          "description" : "Document deletes are published as events",
+          "default" : "true",
+          "type" : "boolean"
+        },
+        "exception-handler" : {
+          "description" : "To let the consumer use a custom ExceptionHandler. Notice if the option bridgeErrorHandler is enabled then this option is not in use. By default the consumer will deal with exceptions, that will be logged at WARN or ERROR level and ignored.",
+          "type" : "string"
+        },
+        "exchange-pattern" : {
+          "description" : "Sets the exchange pattern when the consumer creates an exchange.",
+          "enum" : [ "InOnly", "InOut", "InOptionalOut" ],
+          "type" : "string"
+        },
+        "heartbeat" : {
+          "description" : "How often to send an empty message to keep socket alive in millis",
+          "default" : "30000",
+          "type" : "string"
+        },
+        "lazy-start-producer" : {
+          "description" : "Whether the producer should be started lazy (on the first message). By starting lazy you can use this to allow CamelContext and routes to startup in situations where a producer may otherwise fail during starting and cause the route to fail being started. By deferring this startup to be lazy then the startup failure can be handled during routing messages via Camel's routing error handlers. Beware that when the first message is processed then creating and startin [...]
+          "default" : "false",
+          "type" : "boolean"
+        },
+        "password" : {
+          "description" : "Password for authenticated databases",
+          "type" : "string"
+        },
+        "since" : {
+          "description" : "Start tracking changes immediately after the given update sequence. The default, null, will start monitoring from the latest sequence.",
+          "type" : "string"
+        },
+        "style" : {
+          "description" : "Specifies how many revisions are returned in the changes array. The default, main_only, will only return the current winning revision; all_docs will return all leaf revisions (including conflicts and deleted former conflicts.)",
+          "default" : "main_only",
+          "enum" : [ "all_docs", "main_only" ],
... 31309 lines suppressed ...