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 2009/08/27 10:59:20 UTC

svn commit: r808328 - in /camel/trunk/camel-core/src: main/java/org/apache/camel/ main/java/org/apache/camel/impl/ main/java/org/apache/camel/management/ main/java/org/apache/camel/management/mbean/ test/java/org/apache/camel/management/

Author: davsclaus
Date: Thu Aug 27 08:59:18 2009
New Revision: 808328

URL: http://svn.apache.org/viewvc?rev=808328&view=rev
Log:
CAMEL-1933: Overhaul of JMX. Routes can now be started/stopped.

Added:
    camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRouteStopAndStartTest.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRouteStopTest.java
      - copied, changed from r808276, camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRegisterRouteTest.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/CamelContext.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/Route.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultRoute.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/RouteService.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementStrategy.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedRoute.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedUnregisterCamelContextTest.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/CamelContext.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/CamelContext.java?rev=808328&r1=808327&r2=808328&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/CamelContext.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/CamelContext.java Thu Aug 27 08:59:18 2009
@@ -258,18 +258,53 @@
      *
      * @param route the route to start
      * @throws Exception is thrown if the route could not be started for whatever reason
+     * @deprecated will be removed in Camel 2.2
      */
     void startRoute(RouteDefinition route) throws Exception;
 
     /**
+     * Starts the given route if it has been previously stopped
+     *
+     * @param routeId the route id
+     * @throws Exception is thrown if the route could not be started for whatever reason
+     */
+    void startRoute(String routeId) throws Exception;
+
+    /**
      * Stops the given route. It will remain in the list of route definitions return by {@link #getRouteDefinitions()}
      * unless you use the {@link #removeRouteDefinitions(java.util.Collection)}
      *
      * @param route the route to stop
      * @throws Exception is thrown if the route could not be stopped for whatever reason
+     * @deprecated will be removed in Camel 2.2
      */
     void stopRoute(RouteDefinition route) throws Exception;
 
+    /**
+     * Stops the given route. It will remain in the list of route definitions return by {@link #getRouteDefinitions()}
+     * unless you use the {@link #removeRouteDefinitions(java.util.Collection)}
+     *
+     * @param routeId the route id
+     * @throws Exception is thrown if the route could not be stopped for whatever reason
+     */
+    void stopRoute(String routeId) throws Exception;
+
+    /**
+     * Returns the current status of the given route
+     *
+     * @param routeId the route id
+     * @return the status for the route
+     */
+    ServiceStatus getRouteStatus(String routeId);
+
+    /**
+     * Returns the current status of the given route
+     *
+     * @param route the route
+     * @return the status for the route
+     * @deprecated will be removed in Camel 2.2
+     */
+    ServiceStatus getRouteStatus(RouteDefinition route);
 
     // Properties
     //-----------------------------------------------------------------------
@@ -431,23 +466,6 @@
     FactoryFinder getFactoryFinder(String path) throws NoFactoryAvailableException;
 
     /**
-     * Returns the current status of the given route
-     *
-     * @param routeId the route id
-     * @return the status for the route
-     */
-    ServiceStatus getRouteStatus(String routeId);
-
-    /**
-     * Returns the current status of the given route
-     *
-     * @param route the route
-     * @return the status for the route
-     * @deprecated will be removed in Camel 2.2
-     */
-    ServiceStatus getRouteStatus(RouteDefinition route);
-
-    /**
      * Returns the class resolver to be used for loading/lookup of classes.
      *
      * @return the resolver

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/Route.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/Route.java?rev=808328&r1=808327&r2=808328&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/Route.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/Route.java Thu Aug 27 08:59:18 2009
@@ -53,15 +53,26 @@
 
     /**
      * This property map is used to associate information about
-     * the route. Gets all tbe services for this routes
+     * the route. Gets all the services for this routes
+     * </p>
+     * This implementation is used for initiali
      *
      * @return the services
      * @throws Exception is thrown in case of error
+     * @deprecated will be removed in Camel 2.2
      */
     List<Service> getServicesForRoute() throws Exception;
 
     /**
-     * Returns the additional services required for this particular route
+     * A strategy callback allowing special initialization when services is starting.
+     *
+     * @param services the service
+     * @throws Exception is thrown in case of error
+     */
+    void onStartingServices(List<Service> services) throws Exception;
+
+    /**
+     * Returns the services for this particular route
      */
     List<Service> getServices();
 

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java?rev=808328&r1=808327&r2=808328&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java Thu Aug 27 08:59:18 2009
@@ -597,6 +597,13 @@
         startRouteService(routeService);
     }
 
+    public synchronized void startRoute(String routeId) throws Exception {
+        RouteService routeService = routeServices.get(routeId);
+        if (routeService != null) {
+            routeService.start();
+        }
+    }
+
     public void stopRoute(RouteDefinition route) throws Exception {
         stopRoute(route.idOrCreate(nodeIdFactory));
     }
@@ -605,7 +612,7 @@
      * Stops the route denoted by the given RouteType id
      */
     public synchronized void stopRoute(String key) throws Exception {
-        RouteService routeService = routeServices.remove(key);
+        RouteService routeService = routeServices.get(key);
         if (routeService != null) {
             routeService.stop();
         }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultRoute.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultRoute.java?rev=808328&r1=808327&r2=808328&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultRoute.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultRoute.java Thu Aug 27 08:59:18 2009
@@ -86,6 +86,10 @@
         return servicesForRoute;
     }
 
+    public void onStartingServices(List<Service> services) throws Exception {
+        addServices(services);
+    }
+
     public List<Service> getServices() {
         return services;
     }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/RouteService.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/RouteService.java?rev=808328&r1=808327&r2=808328&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/RouteService.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/RouteService.java Thu Aug 27 08:59:18 2009
@@ -29,6 +29,8 @@
 import org.apache.camel.spi.RouteContext;
 import org.apache.camel.util.EventHelper;
 import org.apache.camel.util.ServiceHelper;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 
 /**
  * Represents the runtime objects for a given {@link RouteDefinition} so that it can be stopped independently
@@ -38,6 +40,8 @@
  */
 public class RouteService extends ServiceSupport {
 
+    private static final Log LOG = LogFactory.getLog(RouteService.class);
+
     private final DefaultCamelContext camelContext;
     private final RouteDefinition routeDefinition;
     private final List<RouteContext> routeContexts;
@@ -80,12 +84,19 @@
         }
 
         for (Route route : routes) {
-            List<Service> services = route.getServicesForRoute();
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Starting route: " + route);
+            }
+
+            List<Service> services = route.getServices();
+
+            // callback that we are staring these services
+            route.onStartingServices(services);
 
             // gather list of services to start as we need to start child services as well
             List<Service> list = new ArrayList<Service>();
             for (Service service : services) {
-                doGetServiesToStart(list, service);
+                doGetChildServies(list, service);
             }
             startChildService(list);
 
@@ -94,37 +105,26 @@
         }
     }
 
-    /**
-     * Need to recursive start child services for routes
-     */
-    private void doGetServiesToStart(List<Service> services, Service service) throws Exception {
-        services.add(service);
-
-        if (service instanceof Navigate) {
-            Navigate<?> nav = (Navigate<?>) service;
-            if (nav.hasNext()) {
-                List<?> children = nav.next();
-                for (Object child : children) {
-                    if (child instanceof Service) {
-                        doGetServiesToStart(services, (Service) child);
-                    }
-                }
-            }
-        }
-    }
-
     protected void doStop() throws Exception {
         for (LifecycleStrategy strategy : camelContext.getLifecycleStrategies()) {
             strategy.onRoutesRemove(routes);
         }
 
-        // do not stop child services as in doStart
-        // as route.getServicesForRoute() will restart
-        // already stopped services, so we end up starting
-        // stuff when we stop.
-
-        // fire events
         for (Route route : routes) {
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Stopping route: " + route);
+            }
+            // getServices will not add services again
+            List<Service> services = route.getServices();
+
+            // gather list of services to stop as we need to start child services as well
+            List<Service> list = new ArrayList<Service>();
+            for (Service service : services) {
+                doGetChildServies(list, service);
+            }
+            stopChildService(list);
+
+            // fire event
             EventHelper.notifyRouteStopped(camelContext, route);
         }
 
@@ -141,4 +141,33 @@
         }
     }
 
+    protected void stopChildService(List<Service> services) throws Exception {
+        for (Service service : services) {
+            for (LifecycleStrategy strategy : camelContext.getLifecycleStrategies()) {
+                strategy.onServiceRemove(camelContext, service);
+            }
+            ServiceHelper.stopService(service);
+            removeChildService(service);
+        }
+    }
+
+    /**
+     * Need to recursive start child services for routes
+     */
+    private static void doGetChildServies(List<Service> services, Service service) throws Exception {
+        services.add(service);
+
+        if (service instanceof Navigate) {
+            Navigate<?> nav = (Navigate<?>) service;
+            if (nav.hasNext()) {
+                List<?> children = nav.next();
+                for (Object child : children) {
+                    if (child instanceof Service) {
+                        doGetChildServies(services, (Service) child);
+                    }
+                }
+            }
+        }
+    }
+
 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementStrategy.java?rev=808328&r1=808327&r2=808328&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementStrategy.java Thu Aug 27 08:59:18 2009
@@ -94,28 +94,28 @@
         return false;
     }
 
-    public void manageObject(Object o) throws Exception {
+    public void manageObject(Object managedObject) throws Exception {
         // noop
     }
 
-    public void manageNamedObject(Object o, Object o1) throws Exception {
+    public void manageNamedObject(Object managedObject, Object preferedName) throws Exception {
         // noop
     }
 
-    public <T> T getManagedObjectName(Object o, String s, Class<T> tClass) throws Exception {
+    public <T> T getManagedObjectName(Object managedObject, String customName, Class<T> nameType) throws Exception {
         // noop
         return null;
     }
 
-    public void unmanageObject(Object o) throws Exception {
+    public void unmanageObject(Object managedObject) throws Exception {
         // noop
     }
 
-    public void unmanageNamedObject(Object o) throws Exception {
+    public void unmanageNamedObject(Object name) throws Exception {
         // noop
     }
 
-    public boolean isManaged(Object o, Object o1) {
+    public boolean isManaged(Object managedObject, Object name) {
         // noop
         return false;
     }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedRoute.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedRoute.java?rev=808328&r1=808327&r2=808328&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedRoute.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedRoute.java Thu Aug 27 08:59:18 2009
@@ -68,7 +68,7 @@
         return ep != null ? ep.getEndpointUri() : VALUE_UNKNOWN;
     }
 
-    @ManagedAttribute(description = "Route state")
+    @ManagedAttribute(description = "Route State")
     public String getState() {
         // must use String type to be sure remote JMX can read the attribute without requiring Camel classes.
         ServiceStatus status = context.getRouteStatus(route.getId());
@@ -86,11 +86,11 @@
 
     @ManagedOperation(description = "Start Route")
     public void start() throws Exception {
-        throw new IllegalArgumentException("Start not supported");
+        context.startRoute(getRouteId());
     }
 
     @ManagedOperation(description = "Stop Route")
     public void stop() throws Exception {
-        throw new IllegalArgumentException("Stop not supported");
+        context.stopRoute(getRouteId());
     }
 }

Added: camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRouteStopAndStartTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRouteStopAndStartTest.java?rev=808328&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRouteStopAndStartTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRouteStopAndStartTest.java Thu Aug 27 08:59:18 2009
@@ -0,0 +1,102 @@
+/**
+ * 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.ContextTestSupport;
+import org.apache.camel.Exchange;
+import org.apache.camel.ServiceStatus;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+/**
+ * @version $Revision$
+ */
+public class ManagedRouteStopAndStartTest extends ContextTestSupport {
+
+    @Override
+    protected void setUp() throws Exception {
+        deleteDirectory("target/managed");
+        super.setUp();
+    }
+
+    public void testStopAndStartRoute() throws Exception {
+        MBeanServer mbeanServer = context.getManagementStrategy().getManagementAgent().getMBeanServer();
+        ObjectName on = getRouteObjectName(mbeanServer);
+
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedBodiesReceived("Hello World");
+
+        template.sendBodyAndHeader("file://target/managed", "Hello World", Exchange.FILE_NAME, "hello.txt");
+
+        assertMockEndpointsSatisfied();
+
+        // should be started
+        String state = (String) mbeanServer.getAttribute(on, "State");
+        assertEquals("Should be started", ServiceStatus.Started.name(), state);
+
+        // stop
+        mbeanServer.invoke(on, "stop", null, null);
+
+        state = (String) mbeanServer.getAttribute(on, "State");
+        assertEquals("Should be stopped", ServiceStatus.Stopped.name(), state);
+
+        mock.reset();
+        mock.expectedBodiesReceived("Bye World");
+        // wait 3 seconds while route is stopped to verify that file was not consumed
+        mock.setResultWaitTime(3000);
+
+        template.sendBodyAndHeader("file://target/managed", "Bye World", Exchange.FILE_NAME, "bye.txt");
+
+        // route is stopped so we do not get the file
+        mock.assertIsNotSatisfied();
+
+        // prepare mock for starting route
+        mock.reset();
+        mock.expectedBodiesReceived("Bye World");
+
+        // start
+        mbeanServer.invoke(on, "start", null, null);
+
+        state = (String) mbeanServer.getAttribute(on, "State");
+        assertEquals("Should be started", ServiceStatus.Started.name(), state);
+
+        // this time the file is consumed
+        mock.assertIsSatisfied();
+    }
+
+    private static ObjectName getRouteObjectName(MBeanServer mbeanServer) throws Exception {
+        Set<ObjectName> set = mbeanServer.queryNames(new ObjectName("*:type=routes,*"), null);
+        assertEquals(1, set.size());
+
+        return set.iterator().next();
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("file://target/managed").to("mock:result");
+            }
+        };
+    }
+
+}

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRouteStopAndStartTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRouteStopAndStartTest.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRouteStopTest.java (from r808276, camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRegisterRouteTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRouteStopTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRouteStopTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRegisterRouteTest.java&r1=808276&r2=808328&rev=808328&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRegisterRouteTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedRouteStopTest.java Thu Aug 27 08:59:18 2009
@@ -27,9 +27,14 @@
 /**
  * @version $Revision$
  */
-public class ManagedRegisterRouteTest extends ContextTestSupport {
+public class ManagedRouteStopTest extends ContextTestSupport {
+
+    public void testStopRoute() throws Exception {
+        // fire a message to get it running
+        getMockEndpoint("mock:result").expectedMessageCount(1);
+        template.sendBody("direct:start", "Hello World");
+        assertMockEndpointsSatisfied();
 
-    public void testRoutes() throws Exception {
         MBeanServer mbeanServer = context.getManagementStrategy().getManagementAgent().getMBeanServer();
 
         Set<ObjectName> set = mbeanServer.queryNames(new ObjectName("*:type=routes,*"), null);
@@ -48,8 +53,7 @@
         String state = (String) mbeanServer.getAttribute(on, "State");
         assertEquals("Should be started", ServiceStatus.Started.name(), state);
 
-        // stop the route
-        context.stopRoute(context.getRouteDefinitions().get(0));
+        mbeanServer.invoke(on, "stop", null, null);
 
         registered = mbeanServer.isRegistered(on);
         assertEquals("Should be registered", true, registered);
@@ -69,4 +73,4 @@
         };
     }
 
-}
+}
\ No newline at end of file

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedUnregisterCamelContextTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedUnregisterCamelContextTest.java?rev=808328&r1=808327&r2=808328&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedUnregisterCamelContextTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/management/ManagedUnregisterCamelContextTest.java Thu Aug 27 08:59:18 2009
@@ -20,8 +20,8 @@
 import javax.management.ObjectName;
 
 import org.apache.camel.CamelContext;
-import org.apache.camel.TestSupport;
 import org.apache.camel.ServiceStatus;
+import org.apache.camel.TestSupport;
 import org.apache.camel.impl.DefaultCamelContext;
 
 /**