You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by da...@apache.org on 2023/12/17 11:26:55 UTC

(camel-spring-boot) branch main updated (aa62a6e276b -> e08242c0a99)

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

davsclaus pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/camel-spring-boot.git


    from aa62a6e276b CAMEL-20242: camel-core: RouteController health check to be DOWN during starting routes. Supervising route controller option to be DOWN during restarting phase.
     new 2799c318c61 Make spring core work
     new e08242c0a99 CAMEL-20243: camel-main - Move route controller options into its own group

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 core/camel-spring-boot-xml/pom.xml                 |   2 -
 core/camel-spring-boot/pom.xml                     |  18 +-
 .../src/main/docs/spring-boot.json                 | 169 ++++++++--------
 .../spring/boot/CamelConfigurationProperties.java  | 101 ----------
 ...upervisingRouteControllerAutoConfiguration.java |  79 ++++++++
 .../SupervisingRouteControllerConfiguration.java   | 224 +++++++++++++++++++++
 ...rk.boot.autoconfigure.AutoConfiguration.imports |   1 +
 .../SupervisingRouteControllerRestartTest.java     |  10 +-
 .../boot/SupervisingRouteControllerTest.java       |  10 +-
 .../endpoint/CamelRouteControllerEndpointTest.java |   2 +-
 10 files changed, 411 insertions(+), 205 deletions(-)
 create mode 100644 core/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/routecontroller/SupervisingRouteControllerAutoConfiguration.java
 create mode 100644 core/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/routecontroller/SupervisingRouteControllerConfiguration.java


(camel-spring-boot) 02/02: CAMEL-20243: camel-main - Move route controller options into its own group

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e08242c0a9920c6fbb7ec1adf7f96161a082d729
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Sun Dec 17 11:57:52 2023 +0100

    CAMEL-20243: camel-main - Move route controller options into its own group
---
 .../src/main/docs/spring-boot.json                 |  87 ++++++++
 .../spring/boot/CamelConfigurationProperties.java  | 101 ----------
 ...upervisingRouteControllerAutoConfiguration.java |  79 ++++++++
 .../SupervisingRouteControllerConfiguration.java   | 224 +++++++++++++++++++++
 ...rk.boot.autoconfigure.AutoConfiguration.imports |   1 +
 .../SupervisingRouteControllerRestartTest.java     |  10 +-
 .../boot/SupervisingRouteControllerTest.java       |  10 +-
 .../endpoint/CamelRouteControllerEndpointTest.java |   2 +-
 8 files changed, 404 insertions(+), 110 deletions(-)

diff --git a/core/camel-spring-boot/src/main/docs/spring-boot.json b/core/camel-spring-boot/src/main/docs/spring-boot.json
index bf93927937f..fa156746649 100644
--- a/core/camel-spring-boot/src/main/docs/spring-boot.json
+++ b/core/camel-spring-boot/src/main/docs/spring-boot.json
@@ -94,6 +94,11 @@
       "sourceType": "org.apache.camel.spring.boot.LanguageConfigurationProperties",
       "sourceMethod": "getCustomizer()"
     },
+    {
+      "name": "camel.routecontroller",
+      "type": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration",
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration"
+    },
     {
       "name": "camel.routetemplate",
       "type": "org.apache.camel.spring.boot.routetemplate.CamelRouteTemplateConfigurationProperties",
@@ -572,6 +577,88 @@
       "sourceType": "org.apache.camel.spring.boot.LanguageConfigurationProperties",
       "defaultValue": true
     },
+    {
+      "name": "camel.routecontroller.back-off-delay",
+      "type": "java.lang.Long",
+      "description": "Backoff delay in millis when restarting a route that failed to startup.",
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration",
+      "defaultValue": 2000
+    },
+    {
+      "name": "camel.routecontroller.back-off-max-attempts",
+      "type": "java.lang.Long",
+      "description": "Backoff maximum number of attempts to restart a route that failed to startup. When this threshold has been exceeded then the controller will give up attempting to restart the route, and the route will remain as stopped.",
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration",
+      "defaultValue": 0
+    },
+    {
+      "name": "camel.routecontroller.back-off-max-delay",
+      "type": "java.lang.Long",
+      "description": "Backoff maximum delay in millis when restarting a route that failed to startup.",
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration",
+      "defaultValue": 0
+    },
+    {
+      "name": "camel.routecontroller.back-off-max-elapsed-time",
+      "type": "java.lang.Long",
+      "description": "Backoff maximum elapsed time in millis, after which the backoff should be considered exhausted and no more attempts should be made.",
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration",
+      "defaultValue": 0
+    },
+    {
+      "name": "camel.routecontroller.back-off-multiplier",
+      "type": "java.lang.Double",
+      "description": "Backoff multiplier to use for exponential backoff. This is used to extend the delay between restart attempts.",
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration",
+      "defaultValue": 1
+    },
+    {
+      "name": "camel.routecontroller.enabled",
+      "type": "java.lang.Boolean",
+      "description": "To enable using supervising route controller which allows Camel to startup and then the controller takes care of starting the routes in a safe manner. This can be used when you want to startup Camel despite a route may otherwise fail fast during startup and cause Camel to fail to startup as well. By delegating the route startup to the supervising route controller then it manages the startup using a background thread. The controller allows to be configured with vario [...]
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration",
+      "defaultValue": false
+    },
+    {
+      "name": "camel.routecontroller.exclude-routes",
+      "type": "java.lang.String",
+      "description": "Pattern for filtering routes to be included as supervised. The pattern is matching on route id, and endpoint uri for the route. Multiple patterns can be separated by comma. For example to include all kafka routes, you can say <tt>kafka:*<\/tt>. And to include routes with specific route ids <tt>myRoute,myOtherRoute<\/tt>. The pattern supports wildcards and uses the matcher from org.apache.camel.support.PatternHelper#matchPattern.",
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration"
+    },
+    {
+      "name": "camel.routecontroller.include-routes",
+      "type": "java.lang.String",
+      "description": "Pattern for filtering routes to be excluded as supervised. The pattern is matching on route id, and endpoint uri for the route. Multiple patterns can be separated by comma. For example to exclude all JMS routes, you can say <tt>jms:*<\/tt>. And to exclude routes with specific route ids <tt>mySpecialRoute,myOtherSpecialRoute<\/tt>. The pattern supports wildcards and uses the matcher from org.apache.camel.support.PatternHelper#matchPattern.",
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration"
+    },
+    {
+      "name": "camel.routecontroller.initial-delay",
+      "type": "java.lang.Long",
+      "description": "Initial delay in milli seconds before the route controller starts, after CamelContext has been started.",
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration",
+      "defaultValue": 0
+    },
+    {
+      "name": "camel.routecontroller.thread-pool-size",
+      "type": "java.lang.Integer",
+      "description": "The number of threads used by the route controller scheduled thread pool that are used for restarting routes. The pool uses 1 thread by default, but you can increase this to allow the controller to concurrently attempt to restart multiple routes in case more than one route has problems starting.",
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration",
+      "defaultValue": 1
+    },
+    {
+      "name": "camel.routecontroller.unhealthy-on-exhausted",
+      "type": "java.lang.Boolean",
+      "description": "Whether to mark the route as unhealthy (down) when all restarting attempts (backoff) have failed and the route is not successfully started and the route manager is giving up. Setting this to true allows health checks to know about this and can report the Camel application as DOWN. The default is false.",
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration",
+      "defaultValue": false
+    },
+    {
+      "name": "camel.routecontroller.unhealthy-on-restarting",
+      "type": "java.lang.Boolean",
+      "description": "Whether to mark the route as unhealthy (down) when the route failed to initially start, and is being controlled for restarting (backoff). Setting this to true allows health checks to know about this and can report the Camel application as DOWN. The default is false.",
+      "sourceType": "org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerConfiguration",
+      "defaultValue": false
+    },
     {
       "name": "camel.routetemplate.config",
       "type": "java.util.List<java.util.Map<java.lang.String,java.lang.String>>",
diff --git a/core/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/CamelConfigurationProperties.java b/core/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/CamelConfigurationProperties.java
index f774fa4afa0..a631a4dd525 100644
--- a/core/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/CamelConfigurationProperties.java
+++ b/core/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/CamelConfigurationProperties.java
@@ -806,107 +806,6 @@ public class CamelConfigurationProperties extends DefaultConfigurationProperties
      */
     private String javaRoutesExcludePattern;
 
-    /**
-     * To enable using supervising route controller which allows Camel to startup
-     * and then the controller takes care of starting the routes in a safe manner.
-     *
-     * This can be used when you want to startup Camel despite a route may otherwise
-     * fail fast during startup and cause Camel to fail to startup as well. By delegating
-     * the route startup to the supervising route controller then it manages the startup
-     * using a background thread. The controller allows to be configured with various
-     * settings to attempt to restart failing routes.
-     */
-    boolean routeControllerSuperviseEnabled;
-
-    /**
-     * The number of threads used by the route controller scheduled thread pool that are used for restarting
-     * routes. The pool uses 1 thread by default, but you can increase this to allow the controller
-     * to concurrently attempt to restart multiple routes in case more than one route has problems
-     * starting.
-     */
-    int routeControllerThreadPoolSize = 1;
-
-    /**
-     * Initial delay in milli seconds before the route controller starts, after
-     * CamelContext has been started.
-     */
-    long routeControllerInitialDelay;
-
-    /**
-     * Backoff delay in millis when restarting a route that failed to startup.
-     */
-    long routeControllerBackOffDelay = 2000;
-
-    /**
-     * Backoff maximum delay in millis when restarting a route that failed to startup.
-     */
-    long routeControllerBackOffMaxDelay;
-
-    /**
-     * Backoff maximum elapsed time in millis, after which the backoff should be considered
-     * exhausted and no more attempts should be made.
-     */
-    long routeControllerBackOffMaxElapsedTime;
-
-    /**
-     * Backoff maximum number of attempts to restart a route that failed to startup.
-     * When this threshold has been exceeded then the controller will give up
-     * attempting to restart the route, and the route will remain as stopped.
-     */
-    long routeControllerBackOffMaxAttempts;
-
-    /**
-     * Backoff multiplier to use for exponential backoff. This is used to extend the delay
-     * between restart attempts.
-     */
-    double routeControllerBackOffMultiplier = 1.0;
-
-    /**
-     * Pattern for filtering routes to be excluded as supervised.
-     *
-     * The pattern is matching on route id, and endpoint uri for the route.
-     * Multiple patterns can be separated by comma.
-     *
-     * For example to exclude all JMS routes, you can say <tt>jms:*</tt>.
-     * And to exclude routes with specific route ids <tt>mySpecialRoute,myOtherSpecialRoute</tt>.
-     * The pattern supports wildcards and uses the matcher from
-     * org.apache.camel.support.PatternHelper#matchPattern.
-     */
-    String routeControllerIncludeRoutes;
-
-    /**
-     * Pattern for filtering routes to be included as supervised.
-     *
-     * The pattern is matching on route id, and endpoint uri for the route.
-     * Multiple patterns can be separated by comma.
-     *
-     * For example to include all kafka routes, you can say <tt>kafka:*</tt>.
-     * And to include routes with specific route ids <tt>myRoute,myOtherRoute</tt>.
-     * The pattern supports wildcards and uses the matcher from
-     * org.apache.camel.support.PatternHelper#matchPattern.
-     */
-    String routeControllerExcludeRoutes;
-
-    /**
-     * Whether to mark the route as unhealthy (down) when all restarting attempts (backoff) have failed and the route is
-     * not successfully started and the route manager is giving up.
-     *
-     * Setting this to true allows health checks to know about this and can report the Camel application as DOWN.
-     *
-     * The default is false.
-     */
-    private boolean routeControllerUnhealthyOnExhausted;
-
-    /**
-     * Whether to mark the route as unhealthy (down) when the route failed to initially start, and is being controlled
-     * for restarting (backoff).
-     *
-     * Setting this to true allows health checks to know about this and can report the Camel application as DOWN.
-     *
-     * The default is false.
-     */
-    private boolean routeControllerUnhealthyOnRestarting;
-
     /**
      * Experimental: Configure the context to be lightweight.
      * This will trigger some optimizations and memory reduction options.
diff --git a/core/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/routecontroller/SupervisingRouteControllerAutoConfiguration.java b/core/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/routecontroller/SupervisingRouteControllerAutoConfiguration.java
new file mode 100644
index 00000000000..dcdc34f9cf0
--- /dev/null
+++ b/core/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/routecontroller/SupervisingRouteControllerAutoConfiguration.java
@@ -0,0 +1,79 @@
+/*
+ * 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.spring.boot.routecontroller;
+
+import org.apache.camel.impl.engine.DefaultSupervisingRouteController;
+import org.apache.camel.spi.RouteController;
+import org.apache.camel.spi.SupervisingRouteController;
+import org.apache.camel.spring.boot.CamelAutoConfiguration;
+import org.springframework.beans.factory.config.ConfigurableBeanFactory;
+import org.springframework.boot.autoconfigure.AutoConfigureBefore;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnMissingBean;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
+import org.springframework.boot.context.properties.EnableConfigurationProperties;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+
+@Configuration(proxyBeanMethods = false)
+@AutoConfigureBefore(CamelAutoConfiguration.class)
+@ConditionalOnProperty(prefix = "camel.routecontroller", name = "enabled")
+@EnableConfigurationProperties(SupervisingRouteControllerConfiguration.class)
+public class SupervisingRouteControllerAutoConfiguration {
+
+    @Bean
+    @ConditionalOnMissingBean
+    public SupervisingRouteController supervisingRouteController(SupervisingRouteControllerConfiguration config) {
+        SupervisingRouteController src = null;
+
+        if (config.isEnabled()) {
+            // switch to supervising route controller
+            src = new DefaultSupervisingRouteController();
+            if (config.getIncludeRoutes() != null) {
+                src.setIncludeRoutes(config.getIncludeRoutes());
+            }
+            if (config.getExcludeRoutes() != null) {
+                src.setExcludeRoutes(config.getExcludeRoutes());
+            }
+            if (config.getThreadPoolSize() > 0) {
+                src.setThreadPoolSize(config.getThreadPoolSize());
+            }
+            if (config.getBackOffDelay() > 0) {
+                src.setBackOffDelay(config.getBackOffDelay());
+            }
+            if (config.getInitialDelay() > 0) {
+                src.setInitialDelay(config.getInitialDelay());
+            }
+            if (config.getBackOffMaxAttempts() > 0) {
+                src.setBackOffMaxAttempts(config.getBackOffMaxAttempts());
+            }
+            if (config.getBackOffMaxDelay() > 0) {
+                src.setBackOffMaxDelay(config.getBackOffDelay());
+            }
+            if (config.getBackOffMaxElapsedTime() > 0) {
+                src.setBackOffMaxElapsedTime(config.getBackOffMaxElapsedTime());
+            }
+            if (config.getBackOffMultiplier() > 0) {
+                src.setBackOffMultiplier(config.getBackOffMultiplier());
+            }
+            src.setUnhealthyOnExhausted(config.isUnhealthyOnExhausted());
+            src.setUnhealthyOnRestarting(config.isUnhealthyOnRestarting());
+        }
+
+        return src;
+    }
+
+}
diff --git a/core/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/routecontroller/SupervisingRouteControllerConfiguration.java b/core/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/routecontroller/SupervisingRouteControllerConfiguration.java
new file mode 100644
index 00000000000..ebe47e0461e
--- /dev/null
+++ b/core/camel-spring-boot/src/main/java/org/apache/camel/spring/boot/routecontroller/SupervisingRouteControllerConfiguration.java
@@ -0,0 +1,224 @@
+/*
+ * 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.spring.boot.routecontroller;
+
+import org.apache.camel.spi.Metadata;
+import org.springframework.boot.context.properties.ConfigurationProperties;
+
+@ConfigurationProperties(prefix = "camel.routecontroller")
+public class SupervisingRouteControllerConfiguration {
+
+    /**
+     * To enable using supervising route controller which allows Camel to startup
+     * and then the controller takes care of starting the routes in a safe manner.
+     *
+     * This can be used when you want to startup Camel despite a route may otherwise
+     * fail fast during startup and cause Camel to fail to startup as well. By delegating
+     * the route startup to the supervising route controller then it manages the startup
+     * using a background thread. The controller allows to be configured with various
+     * settings to attempt to restart failing routes.
+     */
+    boolean enabled;
+
+    /**
+     * The number of threads used by the route controller scheduled thread pool that are used for restarting
+     * routes. The pool uses 1 thread by default, but you can increase this to allow the controller
+     * to concurrently attempt to restart multiple routes in case more than one route has problems
+     * starting.
+     */
+    @Metadata(label = "advanced", defaultValue = "1")
+    int threadPoolSize = 1;
+
+    /**
+     * Initial delay in milli seconds before the route controller starts, after
+     * CamelContext has been started.
+     */
+    long initialDelay;
+
+    /**
+     * Backoff delay in millis when restarting a route that failed to startup.
+     */
+    @Metadata(defaultValue = "2000")
+    long backOffDelay = 2000;
+
+    /**
+     * Backoff maximum delay in millis when restarting a route that failed to startup.
+     */
+    long backOffMaxDelay;
+
+    /**
+     * Backoff maximum elapsed time in millis, after which the backoff should be considered
+     * exhausted and no more attempts should be made.
+     */
+    long backOffMaxElapsedTime;
+
+    /**
+     * Backoff maximum number of attempts to restart a route that failed to startup.
+     * When this threshold has been exceeded then the controller will give up
+     * attempting to restart the route, and the route will remain as stopped.
+     */
+    long backOffMaxAttempts;
+
+    /**
+     * Backoff multiplier to use for exponential backoff. This is used to extend the delay
+     * between restart attempts.
+     */
+    @Metadata(defaultValue = "1.0")
+    double backOffMultiplier = 1.0;
+
+    /**
+     * Pattern for filtering routes to be excluded as supervised.
+     *
+     * The pattern is matching on route id, and endpoint uri for the route.
+     * Multiple patterns can be separated by comma.
+     *
+     * For example to exclude all JMS routes, you can say <tt>jms:*</tt>.
+     * And to exclude routes with specific route ids <tt>mySpecialRoute,myOtherSpecialRoute</tt>.
+     * The pattern supports wildcards and uses the matcher from
+     * org.apache.camel.support.PatternHelper#matchPattern.
+     */
+    String includeRoutes;
+
+    /**
+     * Pattern for filtering routes to be included as supervised.
+     *
+     * The pattern is matching on route id, and endpoint uri for the route.
+     * Multiple patterns can be separated by comma.
+     *
+     * For example to include all kafka routes, you can say <tt>kafka:*</tt>.
+     * And to include routes with specific route ids <tt>myRoute,myOtherRoute</tt>.
+     * The pattern supports wildcards and uses the matcher from
+     * org.apache.camel.support.PatternHelper#matchPattern.
+     */
+    String excludeRoutes;
+
+    /**
+     * Whether to mark the route as unhealthy (down) when all restarting attempts (backoff) have failed and the route is
+     * not successfully started and the route manager is giving up.
+     *
+     * Setting this to true allows health checks to know about this and can report the Camel application as DOWN.
+     *
+     * The default is false.
+     */
+    private boolean unhealthyOnExhausted;
+
+    /**
+     * Whether to mark the route as unhealthy (down) when the route failed to initially start, and is being controlled
+     * for restarting (backoff).
+     *
+     * Setting this to true allows health checks to know about this and can report the Camel application as DOWN.
+     *
+     * The default is false.
+     */
+    private boolean unhealthyOnRestarting;
+
+    public boolean isEnabled() {
+        return enabled;
+    }
+
+    public void setEnabled(boolean enabled) {
+        this.enabled = enabled;
+    }
+
+    public int getThreadPoolSize() {
+        return threadPoolSize;
+    }
+
+    public void setThreadPoolSize(int threadPoolSize) {
+        this.threadPoolSize = threadPoolSize;
+    }
+
+    public long getInitialDelay() {
+        return initialDelay;
+    }
+
+    public void setInitialDelay(long initialDelay) {
+        this.initialDelay = initialDelay;
+    }
+
+    public long getBackOffDelay() {
+        return backOffDelay;
+    }
+
+    public void setBackOffDelay(long backOffDelay) {
+        this.backOffDelay = backOffDelay;
+    }
+
+    public long getBackOffMaxDelay() {
+        return backOffMaxDelay;
+    }
+
+    public void setBackOffMaxDelay(long backOffMaxDelay) {
+        this.backOffMaxDelay = backOffMaxDelay;
+    }
+
+    public long getBackOffMaxElapsedTime() {
+        return backOffMaxElapsedTime;
+    }
+
+    public void setBackOffMaxElapsedTime(long backOffMaxElapsedTime) {
+        this.backOffMaxElapsedTime = backOffMaxElapsedTime;
+    }
+
+    public long getBackOffMaxAttempts() {
+        return backOffMaxAttempts;
+    }
+
+    public void setBackOffMaxAttempts(long backOffMaxAttempts) {
+        this.backOffMaxAttempts = backOffMaxAttempts;
+    }
+
+    public double getBackOffMultiplier() {
+        return backOffMultiplier;
+    }
+
+    public void setBackOffMultiplier(double backOffMultiplier) {
+        this.backOffMultiplier = backOffMultiplier;
+    }
+
+    public String getIncludeRoutes() {
+        return includeRoutes;
+    }
+
+    public void setIncludeRoutes(String includeRoutes) {
+        this.includeRoutes = includeRoutes;
+    }
+
+    public String getExcludeRoutes() {
+        return excludeRoutes;
+    }
+
+    public void setExcludeRoutes(String excludeRoutes) {
+        this.excludeRoutes = excludeRoutes;
+    }
+
+    public boolean isUnhealthyOnExhausted() {
+        return unhealthyOnExhausted;
+    }
+
+    public void setUnhealthyOnExhausted(boolean unhealthyOnExhausted) {
+        this.unhealthyOnExhausted = unhealthyOnExhausted;
+    }
+
+    public boolean isUnhealthyOnRestarting() {
+        return unhealthyOnRestarting;
+    }
+
+    public void setUnhealthyOnRestarting(boolean unhealthyOnRestarting) {
+        this.unhealthyOnRestarting = unhealthyOnRestarting;
+    }
+}
diff --git a/core/camel-spring-boot/src/main/resources/META-INF/spring/org.springframework.boot.autoconfigure.AutoConfiguration.imports b/core/camel-spring-boot/src/main/resources/META-INF/spring/org.springframework.boot.autoconfigure.AutoConfiguration.imports
index 8532b4fa82e..13a8efb52ca 100644
--- a/core/camel-spring-boot/src/main/resources/META-INF/spring/org.springframework.boot.autoconfigure.AutoConfiguration.imports
+++ b/core/camel-spring-boot/src/main/resources/META-INF/spring/org.springframework.boot.autoconfigure.AutoConfiguration.imports
@@ -30,6 +30,7 @@ org.apache.camel.spring.boot.cloud.CamelCloudServiceChooserAutoConfiguration
 org.apache.camel.spring.boot.cluster.ClusteredRouteControllerAutoConfiguration
 org.apache.camel.spring.boot.debug.CamelDebugAutoConfiguration
 org.apache.camel.spring.boot.properties.PropertiesComponentAutoConfiguration
+org.apache.camel.spring.boot.routecontroller.SupervisingRouteControllerAutoConfiguration
 org.apache.camel.spring.boot.security.CamelSSLAutoConfiguration
 org.apache.camel.spring.boot.threadpool.CamelThreadPoolAutoConfiguration
 org.apache.camel.spring.boot.routetemplate.CamelRouteTemplateAutoConfiguration
diff --git a/core/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerRestartTest.java b/core/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerRestartTest.java
index f0c4bba340b..c0e346434ac 100644
--- a/core/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerRestartTest.java
+++ b/core/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerRestartTest.java
@@ -26,6 +26,7 @@ import org.apache.camel.spring.boot.dummy.DummyComponent;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
 import org.springframework.boot.test.context.SpringBootTest;
 import org.springframework.context.annotation.Bean;
 import org.springframework.context.annotation.Configuration;
@@ -36,6 +37,7 @@ import static org.awaitility.Awaitility.await;
 
 @DirtiesContext
 @CamelSpringBootTest
+@EnableAutoConfiguration
 @SpringBootTest(
     classes = {
         CamelAutoConfiguration.class,
@@ -43,10 +45,10 @@ import static org.awaitility.Awaitility.await;
     },
     properties = {
         "camel.springboot.main-run-controller = true",
-        "camel.springboot.routeControllerSuperviseEnabled = true",
-        "camel.springboot.routeControllerInitialDelay = 500",
-        "camel.springboot.routeControllerBackoffDelay = 1000",
-        "camel.springboot.routeControllerBackoffMaxAttempts = 5",
+        "camel.routecontroller.enabled = true",
+        "camel.routecontroller.initialDelay = 500",
+        "camel.routecontroller.backoffDelay = 1000",
+        "camel.routecontroller.backoffMaxAttempts = 5",
     }
 )
 public class SupervisingRouteControllerRestartTest {
diff --git a/core/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerTest.java b/core/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerTest.java
index c0a158c5432..59eca6011b0 100644
--- a/core/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerTest.java
+++ b/core/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/SupervisingRouteControllerTest.java
@@ -26,6 +26,7 @@ import org.apache.camel.test.AvailablePortFinder;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.Test;
 import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.boot.autoconfigure.EnableAutoConfiguration;
 import org.springframework.boot.test.context.SpringBootTest;
 import org.springframework.context.annotation.Bean;
 import org.springframework.context.annotation.Configuration;
@@ -36,6 +37,7 @@ import static org.awaitility.Awaitility.await;
 
 @DirtiesContext
 @CamelSpringBootTest
+@EnableAutoConfiguration
 @SpringBootTest(
     classes = {
         CamelAutoConfiguration.class,
@@ -44,10 +46,10 @@ import static org.awaitility.Awaitility.await;
     properties = {
         "camel.springboot.routes-include-pattern = false",
         "camel.springboot.main-run-controller = true",
-        "camel.springboot.routeControllerSuperviseEnabled = true",
-        "camel.springboot.routeControllerInitialDelay = 500",
-        "camel.springboot.routeControllerBackoffDelay = 1000",
-        "camel.springboot.routeControllerBackoffMaxAttempts = 5",
+        "camel.routecontroller.enabled = true",
+        "camel.routecontroller.initialDelay = 500",
+        "camel.routecontroller.backoffDelay = 1000",
+        "camel.routecontroller.backoffMaxAttempts = 5",
     }
 )
 public class SupervisingRouteControllerTest {
diff --git a/core/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/actuate/endpoint/CamelRouteControllerEndpointTest.java b/core/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/actuate/endpoint/CamelRouteControllerEndpointTest.java
index 5e6e7390c16..f53690df949 100644
--- a/core/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/actuate/endpoint/CamelRouteControllerEndpointTest.java
+++ b/core/camel-spring-boot/src/test/java/org/apache/camel/spring/boot/actuate/endpoint/CamelRouteControllerEndpointTest.java
@@ -40,7 +40,7 @@ import org.apache.camel.test.spring.junit5.CamelSpringBootTest;
         classes = {CamelAutoConfiguration.class, CamelRouteControllerEndpointAutoConfiguration.class,
                 CamelRoutesEndpointAutoConfiguration.class, ActuatorTestRoute.class},
         properties = {"management.endpoints.web.exposure.include=*",
-                      "camel.springboot.routeControllerSuperviseEnabled=true"})
+                      "camel.routecontroller.enabled=true"})
 public class CamelRouteControllerEndpointTest {
 
     @Autowired


(camel-spring-boot) 01/02: Make spring core work

Posted by da...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 2799c318c61fac88c7643c71946466c891e74f60
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Sun Dec 17 11:25:13 2023 +0100

    Make spring core work
---
 core/camel-spring-boot-xml/pom.xml                 |  2 -
 core/camel-spring-boot/pom.xml                     | 18 ++---
 .../src/main/docs/spring-boot.json                 | 82 ----------------------
 3 files changed, 7 insertions(+), 95 deletions(-)

diff --git a/core/camel-spring-boot-xml/pom.xml b/core/camel-spring-boot-xml/pom.xml
index 2d3789d295f..68901a2d046 100644
--- a/core/camel-spring-boot-xml/pom.xml
+++ b/core/camel-spring-boot-xml/pom.xml
@@ -35,8 +35,6 @@
     <properties>
         <firstVersion>3.9.0</firstVersion>
         <label>spring,microservice</label>
-
-        <camel.osgi.export.pkg />
     </properties>
 
     <dependencies>
diff --git a/core/camel-spring-boot/pom.xml b/core/camel-spring-boot/pom.xml
index 139831b828e..c547518b152 100644
--- a/core/camel-spring-boot/pom.xml
+++ b/core/camel-spring-boot/pom.xml
@@ -35,15 +35,12 @@
     <properties>
         <firstVersion>2.15.0</firstVersion>
         <label>spring,microservice</label>
-
-        <camel.osgi.export.pkg />
     </properties>
 
     <dependencies>
         <dependency>
             <groupId>org.springframework.boot</groupId>
             <artifactId>spring-boot-starter</artifactId>
-            <optional>true</optional>
             <version>${spring-boot-version}</version>
         </dependency>
         <dependency>
@@ -70,6 +67,13 @@
             <optional>true</optional>
             <version>${spring-boot-version}</version>
         </dependency>
+        <!-- Optional Spring web support -->
+        <dependency>
+            <groupId>org.springframework.boot</groupId>
+            <artifactId>spring-boot-starter-web</artifactId>
+            <version>${spring-boot-version}</version>
+            <optional>true</optional>
+        </dependency>
 
         <dependency>
             <groupId>org.apache.camel</groupId>
@@ -96,14 +100,6 @@
             <artifactId>camel-health</artifactId>
         </dependency>
 
-        <!-- Optional Spring web support -->
-        <dependency>
-            <groupId>org.springframework.boot</groupId>
-            <artifactId>spring-boot-starter-web</artifactId>
-            <version>${spring-boot-version}</version>
-            <optional>true</optional>
-        </dependency>
-
         <!-- Testing dependencies -->
         <dependency>
             <groupId>org.apache.camel</groupId>
diff --git a/core/camel-spring-boot/src/main/docs/spring-boot.json b/core/camel-spring-boot/src/main/docs/spring-boot.json
index b9e376152cf..bf93927937f 100644
--- a/core/camel-spring-boot/src/main/docs/spring-boot.json
+++ b/core/camel-spring-boot/src/main/docs/spring-boot.json
@@ -957,88 +957,6 @@
       "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties",
       "defaultValue": 1000
     },
-    {
-      "name": "camel.springboot.route-controller-back-off-delay",
-      "type": "java.lang.Long",
-      "description": "Backoff delay in millis when restarting a route that failed to startup.",
-      "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties",
-      "defaultValue": 2000
-    },
-    {
-      "name": "camel.springboot.route-controller-back-off-max-attempts",
-      "type": "java.lang.Long",
-      "description": "Backoff maximum number of attempts to restart a route that failed to startup. When this threshold has been exceeded then the controller will give up attempting to restart the route, and the route will remain as stopped.",
-      "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties",
-      "defaultValue": 0
-    },
-    {
-      "name": "camel.springboot.route-controller-back-off-max-delay",
-      "type": "java.lang.Long",
-      "description": "Backoff maximum delay in millis when restarting a route that failed to startup.",
-      "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties",
-      "defaultValue": 0
-    },
-    {
-      "name": "camel.springboot.route-controller-back-off-max-elapsed-time",
-      "type": "java.lang.Long",
-      "description": "Backoff maximum elapsed time in millis, after which the backoff should be considered exhausted and no more attempts should be made.",
-      "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties",
-      "defaultValue": 0
-    },
-    {
-      "name": "camel.springboot.route-controller-back-off-multiplier",
-      "type": "java.lang.Double",
-      "description": "Backoff multiplier to use for exponential backoff. This is used to extend the delay between restart attempts.",
-      "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties",
-      "defaultValue": 1
-    },
-    {
-      "name": "camel.springboot.route-controller-exclude-routes",
-      "type": "java.lang.String",
-      "description": "Pattern for filtering routes to be included as supervised. The pattern is matching on route id, and endpoint uri for the route. Multiple patterns can be separated by comma. For example to include all kafka routes, you can say <tt>kafka:*<\/tt>. And to include routes with specific route ids <tt>myRoute,myOtherRoute<\/tt>. The pattern supports wildcards and uses the matcher from org.apache.camel.support.PatternHelper#matchPattern.",
-      "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties"
-    },
-    {
-      "name": "camel.springboot.route-controller-include-routes",
-      "type": "java.lang.String",
-      "description": "Pattern for filtering routes to be excluded as supervised. The pattern is matching on route id, and endpoint uri for the route. Multiple patterns can be separated by comma. For example to exclude all JMS routes, you can say <tt>jms:*<\/tt>. And to exclude routes with specific route ids <tt>mySpecialRoute,myOtherSpecialRoute<\/tt>. The pattern supports wildcards and uses the matcher from org.apache.camel.support.PatternHelper#matchPattern.",
-      "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties"
-    },
-    {
-      "name": "camel.springboot.route-controller-initial-delay",
-      "type": "java.lang.Long",
-      "description": "Initial delay in milli seconds before the route controller starts, after CamelContext has been started.",
-      "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties",
-      "defaultValue": 0
-    },
-    {
-      "name": "camel.springboot.route-controller-supervise-enabled",
-      "type": "java.lang.Boolean",
-      "description": "To enable using supervising route controller which allows Camel to startup and then the controller takes care of starting the routes in a safe manner. This can be used when you want to startup Camel despite a route may otherwise fail fast during startup and cause Camel to fail to startup as well. By delegating the route startup to the supervising route controller then it manages the startup using a background thread. The controller allows to be configured with vario [...]
-      "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties",
-      "defaultValue": false
-    },
-    {
-      "name": "camel.springboot.route-controller-thread-pool-size",
-      "type": "java.lang.Integer",
-      "description": "The number of threads used by the route controller scheduled thread pool that are used for restarting routes. The pool uses 1 thread by default, but you can increase this to allow the controller to concurrently attempt to restart multiple routes in case more than one route has problems starting.",
-      "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties",
-      "defaultValue": 1
-    },
-    {
-      "name": "camel.springboot.route-controller-unhealthy-on-exhausted",
-      "type": "java.lang.Boolean",
-      "description": "Whether to mark the route as unhealthy (down) when all restarting attempts (backoff) have failed and the route is not successfully started and the route manager is giving up. Setting this to true allows health checks to know about this and can report the Camel application as DOWN. The default is false.",
-      "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties",
-      "defaultValue": false
-    },
-    {
-      "name": "camel.springboot.route-controller-unhealthy-on-restarting",
-      "type": "java.lang.Boolean",
-      "description": "Whether to mark the route as unhealthy (down) when the route failed to initially start, and is being controlled for restarting (backoff). Setting this to true allows health checks to know about this and can report the Camel application as DOWN. The default is false.",
-      "sourceType": "org.apache.camel.spring.boot.CamelConfigurationProperties",
-      "defaultValue": false
-    },
     {
       "name": "camel.springboot.route-filter-exclude-pattern",
       "type": "java.lang.String",