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 2022/03/02 09:28:47 UTC

[camel] branch controlbus-fail created (now 7661e26)

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

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


      at 7661e26  CAMEL-17726: camel-controlbus - Add fail action to stop and mark a route as failed. Add stop route with caused exception to mark the route as failed (DOWN).

This branch includes the following new commits:

     new 7661e26  CAMEL-17726: camel-controlbus - Add fail action to stop and mark a route as failed. Add stop route with caused exception to mark the route as failed (DOWN).

The 1 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.


[camel] 01/01: CAMEL-17726: camel-controlbus - Add fail action to stop and mark a route as failed. Add stop route with caused exception to mark the route as failed (DOWN).

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

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

commit 7661e26a10cfc4340ad2bab80c76fe4e1f51d523
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Wed Mar 2 10:27:53 2022 +0100

    CAMEL-17726: camel-controlbus - Add fail action to stop and mark a route as failed. Add stop route with caused exception to mark the route as failed (DOWN).
---
 .../camel/component/controlbus/controlbus.json     |   2 +-
 .../component/controlbus/ControlBusEndpoint.java   |  10 +-
 .../component/controlbus/ControlBusProducer.java   |  13 +++
 .../java/org/apache/camel/spi/RouteController.java |  10 ++
 .../camel/impl/engine/DefaultRouteController.java  |   5 +
 .../engine/DefaultSupervisingRouteController.java  |  13 +++
 .../camel/impl/engine/InternalRouteController.java |  11 +++
 .../impl/lw/LightweightRuntimeCamelContext.java    |   5 +
 .../controlbus/ControlBusFailRouteTest.java        |  65 +++++++++++++
 .../apache/camel/issues/StopAndFailRouteTest.java  |  73 ++++++++++++++
 .../api/management/mbean/ManagedRouteMBean.java    |   3 +
 .../camel/management/mbean/ManagedRoute.java       |  10 ++
 .../management/ManagedRouteStopAndFailTest.java    | 108 +++++++++++++++++++++
 .../modules/ROOT/pages/graceful-shutdown.adoc      |  19 ++++
 14 files changed, 341 insertions(+), 6 deletions(-)

diff --git a/components/camel-controlbus/src/generated/resources/org/apache/camel/component/controlbus/controlbus.json b/components/camel-controlbus/src/generated/resources/org/apache/camel/component/controlbus/controlbus.json
index 64d8185..75b407b 100644
--- a/components/camel-controlbus/src/generated/resources/org/apache/camel/component/controlbus/controlbus.json
+++ b/components/camel-controlbus/src/generated/resources/org/apache/camel/component/controlbus/controlbus.json
@@ -28,7 +28,7 @@
   "properties": {
     "command": { "kind": "path", "displayName": "Command", "group": "producer", "label": "", "required": true, "type": "string", "javaType": "java.lang.String", "enum": [ "route", "language" ], "deprecated": false, "deprecationNote": "", "autowired": false, "secret": false, "description": "Command can be either route or language" },
     "language": { "kind": "path", "displayName": "Language", "group": "producer", "label": "", "required": false, "type": "object", "javaType": "org.apache.camel.spi.Language", "enum": [ "bean", "constant", "el", "exchangeProperty", "file", "groovy", "header", "jsonpath", "mvel", "ognl", "ref", "simple", "spel", "sql", "terser", "tokenize", "xpath", "xquery", "xtokenize" ], "deprecated": false, "autowired": false, "secret": false, "description": "Allows you to specify the name of a Langu [...]
-    "action": { "kind": "parameter", "displayName": "Action", "group": "producer", "label": "", "required": false, "type": "string", "javaType": "java.lang.String", "enum": [ "start", "stop", "suspend", "resume", "restart", "status", "stats" ], "deprecated": false, "autowired": false, "secret": false, "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 [...]
+    "action": { "kind": "parameter", "displayName": "Action", "group": "producer", "label": "", "required": false, "type": "string", "javaType": "java.lang.String", "enum": [ "start", "stop", "fail", "suspend", "resume", "restart", "status", "stats" ], "deprecated": false, "autowired": false, "secret": false, "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 [...]
     "async": { "kind": "parameter", "displayName": "Async", "group": "producer", "label": "", "required": false, "type": "boolean", "javaType": "boolean", "deprecated": false, "autowired": false, "secret": false, "defaultValue": false, "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." },
     "lazyStartProducer": { "kind": "parameter", "displayName": "Lazy Start Producer", "group": "producer", "label": "producer", "required": false, "type": "boolean", "javaType": "boolean", "deprecated": false, "autowired": false, "secret": false, "defaultValue": false, "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 sta [...]
     "loggingLevel": { "kind": "parameter", "displayName": "Logging Level", "group": "producer", "label": "", "required": false, "type": "object", "javaType": "org.apache.camel.LoggingLevel", "enum": [ "TRACE", "DEBUG", "INFO", "WARN", "ERROR", "OFF" ], "deprecated": false, "autowired": false, "secret": false, "defaultValue": "INFO", "description": "Logging level used for logging when task is done, or if any exceptions occurred during processing the task." },
diff --git a/components/camel-controlbus/src/main/java/org/apache/camel/component/controlbus/ControlBusEndpoint.java b/components/camel-controlbus/src/main/java/org/apache/camel/component/controlbus/ControlBusEndpoint.java
index b4c049a..c0584ec 100644
--- a/components/camel-controlbus/src/main/java/org/apache/camel/component/controlbus/ControlBusEndpoint.java
+++ b/components/camel-controlbus/src/main/java/org/apache/camel/component/controlbus/ControlBusEndpoint.java
@@ -48,7 +48,7 @@ public class ControlBusEndpoint extends DefaultEndpoint {
     private Language language;
     @UriParam
     private String routeId;
-    @UriParam(enums = "start,stop,suspend,resume,restart,status,stats")
+    @UriParam(enums = "start,stop,fail,suspend,resume,restart,status,stats")
     private String action;
     @UriParam(defaultValue = "1000")
     private int restartDelay = 1000;
@@ -114,10 +114,10 @@ public class ControlBusEndpoint extends DefaultEndpoint {
      * To denote an action that can be either: start, stop, or status.
      * <p/>
      * 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 defined, then you get statistics for the entire
-     * CamelContext. The restart action will restart the route.
+     * suspend and resume to either suspend or resume a route. 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 defined, then you get statistics for the entire CamelContext. The restart action will restart the route. And
+     * the fail action will stop and mark the route as failed (stopped due to an exception)
      */
     public void setAction(String action) {
         this.action = action;
diff --git a/components/camel-controlbus/src/main/java/org/apache/camel/component/controlbus/ControlBusProducer.java b/components/camel-controlbus/src/main/java/org/apache/camel/component/controlbus/ControlBusProducer.java
index b03c463..9e7ce05 100644
--- a/components/camel-controlbus/src/main/java/org/apache/camel/component/controlbus/ControlBusProducer.java
+++ b/components/camel-controlbus/src/main/java/org/apache/camel/component/controlbus/ControlBusProducer.java
@@ -16,6 +16,8 @@
  */
 package org.apache.camel.component.controlbus;
 
+import java.util.concurrent.RejectedExecutionException;
+
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
@@ -163,6 +165,17 @@ public class ControlBusProducer extends DefaultAsyncProducer {
                 } else if ("stop".equals(action)) {
                     LOG.debug("Stopping route: {}", id);
                     getEndpoint().getCamelContext().getRouteController().stopRoute(id);
+                } else if ("fail".equals(action)) {
+                    LOG.debug("Stopping and failing route: {}", id);
+                    // is there any caused exception from the exchange to mark the route as failed due
+                    Throwable cause = exchange.getException();
+                    if (cause == null) {
+                        cause = exchange.getProperty(Exchange.EXCEPTION_CAUGHT, Throwable.class);
+                    }
+                    if (cause == null) {
+                        cause = new RejectedExecutionException("Route " + id + " is forced stopped and marked as failed");
+                    }
+                    getEndpoint().getCamelContext().getRouteController().stopRoute(id, cause);
                 } else if ("suspend".equals(action)) {
                     LOG.debug("Suspending route: {}", id);
                     getEndpoint().getCamelContext().getRouteController().suspendRoute(id);
diff --git a/core/camel-api/src/main/java/org/apache/camel/spi/RouteController.java b/core/camel-api/src/main/java/org/apache/camel/spi/RouteController.java
index 911f9c2..3f1c4f6 100644
--- a/core/camel-api/src/main/java/org/apache/camel/spi/RouteController.java
+++ b/core/camel-api/src/main/java/org/apache/camel/spi/RouteController.java
@@ -125,6 +125,16 @@ public interface RouteController extends CamelContextAware, StaticService {
     void stopRoute(String routeId) throws Exception;
 
     /**
+     * Stops and marks the given route as failed (health check is DOWN) due to a caused exception.
+     *
+     * @param  routeId   the route id
+     * @param  cause     the exception that is causing this route to be stopped and marked as failed
+     * @throws Exception is thrown if the route could not be stopped for whatever reason
+     * @see              #suspendRoute(String)
+     */
+    void stopRoute(String routeId, Throwable cause) throws Exception;
+
+    /**
      * Stops the given route using {@link org.apache.camel.spi.ShutdownStrategy} with a specified timeout.
      *
      * @param  routeId   the route id
diff --git a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultRouteController.java b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultRouteController.java
index e5401f2..0fc55b7 100644
--- a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultRouteController.java
+++ b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultRouteController.java
@@ -125,6 +125,11 @@ public class DefaultRouteController extends ServiceSupport implements RouteContr
     }
 
     @Override
+    public void stopRoute(String routeId, Throwable cause) throws Exception {
+        getInternalRouteController().stopRoute(routeId, cause);
+    }
+
+    @Override
     public void stopRoute(String routeId, long timeout, TimeUnit timeUnit) throws Exception {
         getInternalRouteController().stopRoute(routeId, timeout, timeUnit);
     }
diff --git a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultSupervisingRouteController.java b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultSupervisingRouteController.java
index a70ace4..545a39c 100644
--- a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultSupervisingRouteController.java
+++ b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/DefaultSupervisingRouteController.java
@@ -265,6 +265,19 @@ public class DefaultSupervisingRouteController extends DefaultRouteController im
     }
 
     @Override
+    public void stopRoute(String routeId, Throwable cause) throws Exception {
+        final Optional<RouteHolder> route = routes.stream().filter(r -> r.getId().equals(routeId)).findFirst();
+
+        if (!route.isPresent()) {
+            // This route is unknown to this controller, apply default behaviour
+            // from super class.
+            super.stopRoute(routeId, cause);
+        } else {
+            doStopRoute(route.get(), true, r -> super.stopRoute(routeId, cause));
+        }
+    }
+
+    @Override
     public void stopRoute(String routeId, long timeout, TimeUnit timeUnit) throws Exception {
         final Optional<RouteHolder> route = routes.stream().filter(r -> r.getId().equals(routeId)).findFirst();
 
diff --git a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/InternalRouteController.java b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/InternalRouteController.java
index 95f378e..97130b5 100644
--- a/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/InternalRouteController.java
+++ b/core/camel-base-engine/src/main/java/org/apache/camel/impl/engine/InternalRouteController.java
@@ -24,6 +24,7 @@ import org.apache.camel.LoggingLevel;
 import org.apache.camel.Route;
 import org.apache.camel.ServiceStatus;
 import org.apache.camel.spi.RouteController;
+import org.apache.camel.spi.RouteError;
 import org.apache.camel.spi.SupervisingRouteController;
 
 /**
@@ -103,6 +104,16 @@ class InternalRouteController implements RouteController {
     }
 
     @Override
+    public void stopRoute(String routeId, Throwable cause) throws Exception {
+        Route route = abstractCamelContext.getRoute(routeId);
+        if (route != null) {
+            abstractCamelContext.stopRoute(routeId);
+            // and mark the route as failed and unhealthy (DOWN)
+            route.setLastError(new DefaultRouteError(RouteError.Phase.STOP, cause, true));
+        }
+    }
+
+    @Override
     public void stopRoute(String routeId, long timeout, TimeUnit timeUnit) throws Exception {
         abstractCamelContext.stopRoute(routeId, timeout, timeUnit);
     }
diff --git a/core/camel-core-engine/src/main/java/org/apache/camel/impl/lw/LightweightRuntimeCamelContext.java b/core/camel-core-engine/src/main/java/org/apache/camel/impl/lw/LightweightRuntimeCamelContext.java
index 1bc5b25..034a1ee 100644
--- a/core/camel-core-engine/src/main/java/org/apache/camel/impl/lw/LightweightRuntimeCamelContext.java
+++ b/core/camel-core-engine/src/main/java/org/apache/camel/impl/lw/LightweightRuntimeCamelContext.java
@@ -2163,6 +2163,11 @@ public class LightweightRuntimeCamelContext implements ExtendedCamelContext, Cat
             }
 
             @Override
+            public void stopRoute(String routeId, Throwable cause) throws Exception {
+                throw new UnsupportedOperationException();
+            }
+
+            @Override
             public void stopRoute(String routeId, long timeout, TimeUnit timeUnit) throws Exception {
                 throw new UnsupportedOperationException();
             }
diff --git a/core/camel-core/src/test/java/org/apache/camel/component/controlbus/ControlBusFailRouteTest.java b/core/camel-core/src/test/java/org/apache/camel/component/controlbus/ControlBusFailRouteTest.java
new file mode 100644
index 0000000..6a09033
--- /dev/null
+++ b/core/camel-core/src/test/java/org/apache/camel/component/controlbus/ControlBusFailRouteTest.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.component.controlbus;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Route;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.spi.RouteError;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import static org.awaitility.Awaitility.await;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class ControlBusFailRouteTest extends ContextTestSupport {
+
+    @Test
+    public void testControlBusFail() throws Exception {
+        assertEquals("Started", context.getRouteController().getRouteStatus("foo").name());
+
+        template.sendBody("direct:foo", "Hello World");
+
+        // runs async so it can take a little while
+        await().atMost(5, TimeUnit.SECONDS).until(() -> context.getRouteController().getRouteStatus("foo").isStopped());
+
+        Route route = context.getRoute("foo");
+        RouteError re = route.getLastError();
+        Assertions.assertNotNull(re);
+        Assertions.assertTrue(re.isUnhealthy());
+        Assertions.assertEquals(RouteError.Phase.STOP, re.getPhase());
+        Throwable cause = re.getException();
+        Assertions.assertNotNull(cause);
+        Assertions.assertEquals("Forced by Donkey Kong", cause.getMessage());
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                errorHandler(deadLetterChannel("controlbus:route?routeId=current&action=fail&async=true"));
+
+                from("direct:foo").routeId("foo")
+                        .throwException(new IllegalArgumentException("Forced by Donkey Kong"));
+            }
+        };
+    }
+}
diff --git a/core/camel-core/src/test/java/org/apache/camel/issues/StopAndFailRouteTest.java b/core/camel-core/src/test/java/org/apache/camel/issues/StopAndFailRouteTest.java
new file mode 100644
index 0000000..9fb0e26
--- /dev/null
+++ b/core/camel-core/src/test/java/org/apache/camel/issues/StopAndFailRouteTest.java
@@ -0,0 +1,73 @@
+/*
+ * 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.issues;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.Route;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.spi.RouteError;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class StopAndFailRouteTest extends ContextTestSupport {
+
+    @Test
+    public void stopRoute() throws Exception {
+        Route route = context.getRoute("foo");
+        Assertions.assertNull(route.getLastError());
+
+        context.getRouteController().stopRoute("foo");
+
+        assertEquals("Stopped", context.getRouteController().getRouteStatus("foo").name());
+
+        RouteError re = route.getLastError();
+        Assertions.assertNull(re);
+    }
+
+    @Test
+    public void failRoute() throws Exception {
+        Route route = context.getRoute("bar");
+        Assertions.assertNull(route.getLastError());
+
+        Throwable cause = new IllegalArgumentException("Forced");
+        context.getRouteController().stopRoute("bar", cause);
+
+        assertEquals("Stopped", context.getRouteController().getRouteStatus("bar").name());
+
+        RouteError re = route.getLastError();
+        Assertions.assertNotNull(re);
+        Assertions.assertTrue(re.isUnhealthy());
+        Assertions.assertEquals(RouteError.Phase.STOP, re.getPhase());
+        Assertions.assertSame(cause, re.getException());
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:foo").routeId("foo")
+                        .to("mock:foo");
+
+                from("direct:bar").routeId("bar")
+                        .to("mock:bar");
+            }
+        };
+    }
+}
diff --git a/core/camel-management-api/src/main/java/org/apache/camel/api/management/mbean/ManagedRouteMBean.java b/core/camel-management-api/src/main/java/org/apache/camel/api/management/mbean/ManagedRouteMBean.java
index dfd3667..5cf705e 100644
--- a/core/camel-management-api/src/main/java/org/apache/camel/api/management/mbean/ManagedRouteMBean.java
+++ b/core/camel-management-api/src/main/java/org/apache/camel/api/management/mbean/ManagedRouteMBean.java
@@ -89,6 +89,9 @@ public interface ManagedRouteMBean extends ManagedPerformanceCounterMBean {
     @ManagedOperation(description = "Stop route")
     void stop() throws Exception;
 
+    @ManagedOperation(description = "Stop and marks the route as failed (health-check reporting as DOWN)")
+    void stopAndFail() throws Exception;
+
     @ManagedOperation(description = "Stop route (using timeout in seconds)")
     void stop(long timeout) throws Exception;
 
diff --git a/core/camel-management/src/main/java/org/apache/camel/management/mbean/ManagedRoute.java b/core/camel-management/src/main/java/org/apache/camel/management/mbean/ManagedRoute.java
index 8b38164..436bc8f 100644
--- a/core/camel-management/src/main/java/org/apache/camel/management/mbean/ManagedRoute.java
+++ b/core/camel-management/src/main/java/org/apache/camel/management/mbean/ManagedRoute.java
@@ -24,6 +24,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import javax.management.AttributeValueExp;
@@ -290,6 +291,15 @@ public class ManagedRoute extends ManagedPerformanceCounter implements TimerList
     }
 
     @Override
+    public void stopAndFail() throws Exception {
+        if (!context.getStatus().isStarted()) {
+            throw new IllegalArgumentException("CamelContext is not started");
+        }
+        Throwable cause = new RejectedExecutionException("Route " + getRouteId() + " is forced stopped and marked as failed");
+        context.getRouteController().stopRoute(getRouteId(), cause);
+    }
+
+    @Override
     public void stop(long timeout) throws Exception {
         if (!context.getStatus().isStarted()) {
             throw new IllegalArgumentException("CamelContext is not started");
diff --git a/core/camel-management/src/test/java/org/apache/camel/management/ManagedRouteStopAndFailTest.java b/core/camel-management/src/test/java/org/apache/camel/management/ManagedRouteStopAndFailTest.java
new file mode 100644
index 0000000..3e12f90
--- /dev/null
+++ b/core/camel-management/src/test/java/org/apache/camel/management/ManagedRouteStopAndFailTest.java
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.management;
+
+import java.util.Set;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
+import org.apache.camel.Route;
+import org.apache.camel.ServiceStatus;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.spi.RouteError;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.condition.DisabledOnOs;
+import org.junit.jupiter.api.condition.OS;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@DisabledOnOs(OS.AIX)
+public class ManagedRouteStopAndFailTest extends ManagementTestSupport {
+
+    @Test
+    public void testStopAndFailRoute() throws Exception {
+        // fire a message to get it running
+        getMockEndpoint("mock:result").expectedMessageCount(1);
+        template.sendBody("direct:start", "Hello World");
+        assertMockEndpointsSatisfied();
+
+        MBeanServer mbeanServer = getMBeanServer();
+
+        Set<ObjectName> set = mbeanServer.queryNames(new ObjectName("*:type=routes,*"), null);
+        assertEquals(1, set.size());
+
+        ObjectName on = set.iterator().next();
+
+        boolean registered = mbeanServer.isRegistered(on);
+        assertEquals(true, registered, "Should be registered");
+
+        String uri = (String) mbeanServer.getAttribute(on, "EndpointUri");
+        // the route has this starting endpoint uri
+        assertEquals("direct://start", uri);
+
+        // should be started
+        String state = (String) mbeanServer.getAttribute(on, "State");
+        assertEquals(ServiceStatus.Started.name(), state, "Should be started");
+
+        String uptime = (String) mbeanServer.getAttribute(on, "Uptime");
+        assertNotNull(uptime);
+        log.info("Uptime: {}", uptime);
+
+        long uptimeMillis = (Long) mbeanServer.getAttribute(on, "UptimeMillis");
+        assertTrue(uptimeMillis > 0);
+
+        String routeId = (String) mbeanServer.getAttribute(on, "RouteId");
+
+        mbeanServer.invoke(on, "stopAndFail", null, null);
+
+        registered = mbeanServer.isRegistered(on);
+        assertEquals(true, registered, "Should be registered");
+
+        // should be stopped, eg its removed
+        state = (String) mbeanServer.getAttribute(on, "State");
+        assertEquals(ServiceStatus.Stopped.name(), state, "Should be stopped");
+
+        uptime = (String) mbeanServer.getAttribute(on, "Uptime");
+        assertEquals("", uptime);
+
+        uptimeMillis = (Long) mbeanServer.getAttribute(on, "UptimeMillis");
+        assertEquals(0, uptimeMillis);
+
+        Route route = context.getRoute(routeId);
+        RouteError re = route.getLastError();
+        Assertions.assertNotNull(re);
+        Assertions.assertTrue(re.isUnhealthy());
+        Assertions.assertEquals(RouteError.Phase.STOP, re.getPhase());
+        Assertions.assertEquals("Route " + routeId + " is forced stopped and marked as failed", re.getException().getMessage());
+
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start").delayer(10).to("log:foo").to("mock:result");
+            }
+        };
+    }
+
+}
diff --git a/docs/user-manual/modules/ROOT/pages/graceful-shutdown.adoc b/docs/user-manual/modules/ROOT/pages/graceful-shutdown.adoc
index 0110364..cafa4c7 100644
--- a/docs/user-manual/modules/ROOT/pages/graceful-shutdown.adoc
+++ b/docs/user-manual/modules/ROOT/pages/graceful-shutdown.adoc
@@ -161,6 +161,25 @@ camelContext.getRouteController().stopRoute(routeId);
 
 Routes can also be stopped via JMX.
 
+=== Stopping and marking routes as failed due to an exception
+
+It is possible to stop and fail (will do a gracefully shut down) an individual route using
+`stopRoute(routeId, cause)` method as shown:
+
+[source,java]
+----
+Exception cause = ...
+camelContext.getRouteController().stopRoute(routeId, cause);
+----
+
+This will stop the route and then mark the route as failed with the caused exception.
+
+NOTE: The Camel xref:health-check.adoc[Health Check] detect the route as failed and report it as DOWN.
+If the route is manually stopped, then the route is not marked as failed,
+and the xref:health-check.adoc[Health Check] will report the status as UNKNOWN.
+
+Routes can also be stopped and failed via JMX.
+
 == Implementing custom component or ShutdownStrategy
 
 If you develop your own Camel component or want to implement your own