You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by cs...@apache.org on 2011/08/30 00:25:24 UTC

svn commit: r1163044 - in /camel/trunk: camel-core/src/main/java/org/apache/camel/ camel-core/src/main/java/org/apache/camel/impl/ camel-core/src/main/java/org/apache/camel/management/mbean/ camel-core/src/main/java/org/apache/camel/processor/ camel-co...

Author: cschneider
Date: Mon Aug 29 22:25:23 2011
New Revision: 1163044

URL: http://svn.apache.org/viewvc?rev=1163044&view=rev
Log:
CAMEL-4381 Introduce StatefulService, split ServiceSupport. ChildServiceSupport for service with children. Resolve cycle between util and support by moving DefaultTimeoutMap to support

Added:
    camel/trunk/camel-core/src/main/java/org/apache/camel/StatefulService.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/TimeoutMap.java   (contents, props changed)
      - copied, changed from r1161594, camel/trunk/camel-core/src/main/java/org/apache/camel/util/TimeoutMap.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/support/ChildServiceSupport.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/support/DefaultTimeoutMap.java   (contents, props changed)
      - copied, changed from r1161594, camel/trunk/camel-core/src/main/java/org/apache/camel/util/DefaultTimeoutMap.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/support/TimeoutMapEntry.java   (contents, props changed)
      - copied, changed from r1161594, camel/trunk/camel-core/src/main/java/org/apache/camel/util/TimeoutMapEntry.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/support/
    camel/trunk/camel-core/src/test/java/org/apache/camel/support/DefaultTimeoutMapTest.java   (contents, props changed)
      - copied, changed from r1161594, camel/trunk/camel-core/src/test/java/org/apache/camel/util/DefaultTimeoutMapTest.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingConsumerPollStrategy.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/mbean/ManagedComponent.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedEndpoint.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedProcessor.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedService.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ErrorHandlerSupport.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RoutePolicyProcessor.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/processor/aggregate/AggregateProcessor.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/support/ServiceSupport.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/util/DefaultTimeoutMap.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/util/EventHelper.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/util/Ordered.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/util/ServiceHelper.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/util/TimeoutMap.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/util/TimeoutMapEntry.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/processor/Endpoint2MustBeStartedBeforeSendProcessorTest.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/util/DefaultTimeoutMapTest.java
    camel/trunk/components/camel-jms/src/main/java/org/apache/camel/component/jms/reply/CorrelationMap.java
    camel/trunk/components/camel-quickfix/src/test/java/org/apache/camel/component/quickfixj/QuickfixjComponentTest.java
    camel/trunk/components/camel-quickfix/src/test/java/org/apache/camel/component/quickfixj/QuickfixjConsumerTest.java

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/StatefulService.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/StatefulService.java?rev=1163044&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/StatefulService.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/StatefulService.java Mon Aug 29 22:25:23 2011
@@ -0,0 +1,74 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel;
+
+
+public interface StatefulService extends Service, ShutdownableService {
+
+    void suspend() throws Exception;
+
+    void resume() throws Exception;
+
+    /**
+     * Returns the current status
+     */
+    ServiceStatus getStatus();
+
+    /**
+     * @return true if this service has been started
+     */
+    boolean isStarted();
+
+    /**
+     * @return true if this service is being started
+     */
+    boolean isStarting();
+
+    /**
+     * @return true if this service is in the process of stopping
+     */
+    boolean isStopping();
+
+    /**
+     * @return true if this service is stopped
+     */
+    boolean isStopped();
+
+    /**
+     * @return true if this service is in the process of suspending
+     */
+    boolean isSuspending();
+
+    /**
+     * @return true if this service is suspended
+     */
+    boolean isSuspended();
+
+    /**
+     * Helper methods so the service knows if it should keep running.
+     * Returns <tt>false</tt> if the service is being stopped or is stopped.
+     *
+     * @return <tt>true</tt> if the service should continue to run.
+     */
+    boolean isRunAllowed();
+
+    /**
+     * Returns the version of this service
+     */
+    String getVersion();
+
+}

Copied: camel/trunk/camel-core/src/main/java/org/apache/camel/TimeoutMap.java (from r1161594, camel/trunk/camel-core/src/main/java/org/apache/camel/util/TimeoutMap.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/TimeoutMap.java?p2=camel/trunk/camel-core/src/main/java/org/apache/camel/TimeoutMap.java&p1=camel/trunk/camel-core/src/main/java/org/apache/camel/util/TimeoutMap.java&r1=1161594&r2=1163044&rev=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/util/TimeoutMap.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/TimeoutMap.java Mon Aug 29 22:25:23 2011
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.util;
+package org.apache.camel;
 
 /**
  * Represents a map of values which timeout after a period of inactivity.

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/TimeoutMap.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/TimeoutMap.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

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=1163044&r1=1163043&r2=1163044&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 Mon Aug 29 22:25:23 2011
@@ -61,6 +61,7 @@ import org.apache.camel.ServiceStatus;
 import org.apache.camel.ShutdownRoute;
 import org.apache.camel.ShutdownRunningTask;
 import org.apache.camel.StartupListener;
+import org.apache.camel.StatefulService;
 import org.apache.camel.SuspendableService;
 import org.apache.camel.TypeConverter;
 import org.apache.camel.VetoCamelContextStartException;
@@ -296,7 +297,9 @@ public class DefaultCamelContext extends
                         addComponent(name, component);
                         if (isStarted() || isStarting()) {
                             // If the component is looked up after the context is started, lets start it up.
-                            startServices(component);
+                            if (component instanceof Service) {
+                                startService((Service)component);
+                            }
                         }
                     }
                 } catch (Exception e) {
@@ -351,7 +354,7 @@ public class DefaultCamelContext extends
     public Endpoint addEndpoint(String uri, Endpoint endpoint) throws Exception {
         Endpoint oldEndpoint;
 
-        startServices(endpoint);
+        startService(endpoint);
         oldEndpoint = endpoints.remove(getEndpointKey(uri));
         for (LifecycleStrategy strategy : lifecycleStrategies) {
             strategy.onEndpointAdd(endpoint);
@@ -902,7 +905,11 @@ public class DefaultCamelContext extends
         }
 
         // and then ensure service is started (as stated in the javadoc)
-        startServices(object);
+        if (object instanceof Service) {
+            startService((Service)object);
+        } else if (object instanceof Collection<?>) {
+            startServices((Collection)object);
+        }
     }
 
     public boolean hasService(Object object) {
@@ -1184,7 +1191,7 @@ public class DefaultCamelContext extends
         answer.setMaximumCacheSize(maximumCacheSize);
         // start it so its ready to use
         try {
-            startServices(answer);
+            startService(answer);
         } catch (Exception e) {
             throw ObjectHelper.wrapRuntimeCamelException(e);
         }
@@ -1201,7 +1208,7 @@ public class DefaultCamelContext extends
         answer.setMaximumCacheSize(maximumCacheSize);
         // start it so its ready to use
         try {
-            startServices(answer);
+            startService(answer);
         } catch (Exception e) {
             throw ObjectHelper.wrapRuntimeCamelException(e);
         }
@@ -1398,7 +1405,7 @@ public class DefaultCamelContext extends
             log.info("Debugger: " + getDebugger() + " is enabled on CamelContext: " + getName());
             // register this camel context on the debugger
             getDebugger().setCamelContext(this);
-            startServices(getDebugger());
+            startService(getDebugger());
             addInterceptStrategy(new Debug(getDebugger()));
         }
 
@@ -1429,7 +1436,9 @@ public class DefaultCamelContext extends
                     strategy.onServiceAdd(this, service, null);
                 }
             }
-            startServices(notifier);
+            if (notifier instanceof Service) {
+                startService((Service)notifier);
+            }
         }
 
         // must let some bootstrap service be started before we can notify the starting event
@@ -1550,9 +1559,9 @@ public class DefaultCamelContext extends
             if (consumer instanceof SuspendableService) {
                 // consumer could be suspended, which is not reflected in the RouteService status
                 startable = ((SuspendableService) consumer).isSuspended();
-            } else if (consumer instanceof ServiceSupport) {
+            } else if (consumer instanceof StatefulService) {
                 // consumer could be stopped, which is not reflected in the RouteService status
-                startable = ((ServiceSupport) consumer).getStatus().isStartable();
+                startable = ((StatefulService) consumer).getStatus().isStartable();
             } else {
                 // no consumer so use state from route service
                 startable = entry.getValue().getStatus().isStartable();
@@ -1589,7 +1598,11 @@ public class DefaultCamelContext extends
 
         // allow us to do custom work before delegating to service helper
         try {
-            ServiceHelper.stopAndShutdownService(service);
+            if (service instanceof Service) {
+                ServiceHelper.stopAndShutdownService((Service)service);
+            } else if (service instanceof Collection) {
+                ServiceHelper.stopAndShutdownServices((Collection<?>)service);
+            }
         } catch (Throwable e) {
             log.warn("Error occurred while shutting down service: " + service + ". This exception will be ignored.", e);
             // fire event
@@ -1615,14 +1628,7 @@ public class DefaultCamelContext extends
         }
     }
 
-    private void startServices(Object service) throws Exception {
-        // it can be a collection so ensure we look inside it
-        if (service instanceof Collection<?>) {
-            for (Object element : (Collection<?>)service) {
-                startServices(element);
-            }
-        }
-
+    private void startService(Service service) throws Exception {
         // and register startup aware so they can be notified when
         // camel context has been started
         if (service instanceof StartupListener) {
@@ -1630,20 +1636,15 @@ public class DefaultCamelContext extends
             addStartupListener(listener);
         }
 
-        // and then start the service
-        ServiceHelper.startService(service);
+        service.start();
     }
-
-    private void resumeServices(Object service) throws Exception {
-        // it can be a collection so ensure we look inside it
-        if (service instanceof Collection<?>) {
-            for (Object element : (Collection<?>)service) {
-                resumeServices(element);
+    
+    private void startServices(Collection services) throws Exception {
+        for (Object element : services) {
+            if (element instanceof Service) {
+                startService((Service)element);
             }
         }
-
-        // and then start the service
-        ServiceHelper.resumeService(service);
     }
 
     private void stopServices(Object service) throws Exception {
@@ -1900,14 +1901,14 @@ public class DefaultCamelContext extends
 
                 if (resumeOnly && route.supportsSuspension()) {
                     // if we are resuming and the route can be resumed
-                    resumeServices(consumer);
+                    ServiceHelper.resumeService(consumer);
                     log.info("Route: " + route.getId() + " resumed and consuming from: " + endpoint);
                 } else {
                     // when starting we should invoke the lifecycle strategies
                     for (LifecycleStrategy strategy : lifecycleStrategies) {
                         strategy.onServiceAdd(this, consumer, route);
                     }
-                    startServices(consumer);
+                    startService(consumer);
                     log.info("Route: " + route.getId() + " started and consuming from: " + endpoint);
                 }
 

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingConsumerPollStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingConsumerPollStrategy.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingConsumerPollStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultPollingConsumerPollStrategy.java Mon Aug 29 22:25:23 2011
@@ -18,8 +18,8 @@ package org.apache.camel.impl;
 
 import org.apache.camel.Consumer;
 import org.apache.camel.Endpoint;
+import org.apache.camel.StatefulService;
 import org.apache.camel.spi.PollingConsumerPollStrategy;
-import org.apache.camel.support.ServiceSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,8 +44,8 @@ public class DefaultPollingConsumerPollS
 
     public boolean rollback(Consumer consumer, Endpoint endpoint, int retryCounter, Exception e) throws Exception {
         boolean runAllowed = true;
-        if (consumer instanceof ServiceSupport) {
-            runAllowed = ((ServiceSupport) consumer).isRunAllowed();
+        if (consumer instanceof StatefulService) {
+            runAllowed = ((StatefulService) consumer).isRunAllowed();
         }
 
         // only log warn if we are running, otherwise we are just stopping which we should not log the issue in the logs

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=1163044&r1=1163043&r2=1163044&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 Mon Aug 29 22:25:23 2011
@@ -33,7 +33,7 @@ import org.apache.camel.Service;
 import org.apache.camel.model.RouteDefinition;
 import org.apache.camel.spi.LifecycleStrategy;
 import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.ServiceSupport;
+import org.apache.camel.support.ChildServiceSupport;
 import org.apache.camel.util.EventHelper;
 import org.apache.camel.util.ServiceHelper;
 import org.slf4j.Logger;
@@ -45,7 +45,7 @@ import org.slf4j.LoggerFactory;
  *
  * @version 
  */
-public class RouteService extends ServiceSupport {
+public class RouteService extends ChildServiceSupport {
 
     private static final Logger LOG = LoggerFactory.getLogger(RouteService.class);
 

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedComponent.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedComponent.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedComponent.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedComponent.java Mon Aug 29 22:25:23 2011
@@ -18,11 +18,11 @@ package org.apache.camel.management.mbea
 
 import org.apache.camel.Component;
 import org.apache.camel.ServiceStatus;
+import org.apache.camel.StatefulService;
 import org.apache.camel.spi.ManagementStrategy;
 import org.apache.camel.spi.management.ManagedAttribute;
 import org.apache.camel.spi.management.ManagedInstance;
 import org.apache.camel.spi.management.ManagedResource;
-import org.apache.camel.support.ServiceSupport;
 
 /**
  * @version 
@@ -53,8 +53,8 @@ public class ManagedComponent implements
     @ManagedAttribute(description = "Component State")
     public String getState() {
         // must use String type to be sure remote JMX can read the attribute without requiring Camel classes.
-        if (component instanceof ServiceSupport) {
-            ServiceStatus status = ((ServiceSupport) component).getStatus();
+        if (component instanceof StatefulService) {
+            ServiceStatus status = ((StatefulService) component).getStatus();
             // if no status exists then its stopped
             if (status == null) {
                 status = ServiceStatus.Stopped;

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedEndpoint.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedEndpoint.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedEndpoint.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedEndpoint.java Mon Aug 29 22:25:23 2011
@@ -18,11 +18,11 @@ package org.apache.camel.management.mbea
 
 import org.apache.camel.Endpoint;
 import org.apache.camel.ServiceStatus;
+import org.apache.camel.StatefulService;
 import org.apache.camel.spi.ManagementStrategy;
 import org.apache.camel.spi.management.ManagedAttribute;
 import org.apache.camel.spi.management.ManagedInstance;
 import org.apache.camel.spi.management.ManagedResource;
-import org.apache.camel.support.ServiceSupport;
 
 @ManagedResource(description = "Managed Endpoint")
 public class ManagedEndpoint implements ManagedInstance {
@@ -58,8 +58,8 @@ public class ManagedEndpoint implements 
     @ManagedAttribute(description = "Service State")
     public String getState() {
         // must use String type to be sure remote JMX can read the attribute without requiring Camel classes.
-        if (endpoint instanceof ServiceSupport) {
-            ServiceStatus status = ((ServiceSupport) endpoint).getStatus();
+        if (endpoint instanceof StatefulService) {
+            ServiceStatus status = ((StatefulService) endpoint).getStatus();
             // if no status exists then its stopped
             if (status == null) {
                 status = ServiceStatus.Stopped;

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedProcessor.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedProcessor.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedProcessor.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedProcessor.java Mon Aug 29 22:25:23 2011
@@ -21,12 +21,12 @@ import org.apache.camel.ManagementStatis
 import org.apache.camel.Processor;
 import org.apache.camel.Route;
 import org.apache.camel.ServiceStatus;
+import org.apache.camel.StatefulService;
 import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.spi.management.ManagedAttribute;
 import org.apache.camel.spi.management.ManagedInstance;
 import org.apache.camel.spi.management.ManagedOperation;
 import org.apache.camel.spi.management.ManagedResource;
-import org.apache.camel.support.ServiceSupport;
 import org.apache.camel.util.ServiceHelper;
 
 /**
@@ -74,8 +74,8 @@ public class ManagedProcessor extends Ma
     @ManagedAttribute(description = "Processor State")
     public String getState() {
         // must use String type to be sure remote JMX can read the attribute without requiring Camel classes.
-        if (processor instanceof ServiceSupport) {
-            ServiceStatus status = ((ServiceSupport) processor).getStatus();
+        if (processor instanceof StatefulService) {
+            ServiceStatus status = ((StatefulService) processor).getStatus();
             // if no status exists then its stopped
             if (status == null) {
                 status = ServiceStatus.Stopped;

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedService.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedService.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedService.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedService.java Mon Aug 29 22:25:23 2011
@@ -20,13 +20,13 @@ import org.apache.camel.CamelContext;
 import org.apache.camel.Route;
 import org.apache.camel.Service;
 import org.apache.camel.ServiceStatus;
+import org.apache.camel.StatefulService;
 import org.apache.camel.SuspendableService;
 import org.apache.camel.spi.ManagementStrategy;
 import org.apache.camel.spi.management.ManagedAttribute;
 import org.apache.camel.spi.management.ManagedInstance;
 import org.apache.camel.spi.management.ManagedOperation;
 import org.apache.camel.spi.management.ManagedResource;
-import org.apache.camel.support.ServiceSupport;
 
 @ManagedResource(description = "Managed Service")
 public class ManagedService implements ManagedInstance {
@@ -62,8 +62,8 @@ public class ManagedService implements M
     @ManagedAttribute(description = "Service State")
     public String getState() {
         // must use String type to be sure remote JMX can read the attribute without requiring Camel classes.
-        if (service instanceof ServiceSupport) {
-            ServiceStatus status = ((ServiceSupport) service).getStatus();
+        if (service instanceof StatefulService) {
+            ServiceStatus status = ((StatefulService) service).getStatus();
             // if no status exists then its stopped
             if (status == null) {
                 status = ServiceStatus.Stopped;

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ErrorHandlerSupport.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ErrorHandlerSupport.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ErrorHandlerSupport.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/ErrorHandlerSupport.java Mon Aug 29 22:25:23 2011
@@ -28,7 +28,7 @@ import org.apache.camel.model.RouteDefin
 import org.apache.camel.processor.exceptionpolicy.DefaultExceptionPolicyStrategy;
 import org.apache.camel.processor.exceptionpolicy.ExceptionPolicyKey;
 import org.apache.camel.processor.exceptionpolicy.ExceptionPolicyStrategy;
-import org.apache.camel.support.ServiceSupport;
+import org.apache.camel.support.ChildServiceSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -37,7 +37,7 @@ import org.slf4j.LoggerFactory;
  *
  * @version 
  */
-public abstract class ErrorHandlerSupport extends ServiceSupport implements ErrorHandler {
+public abstract class ErrorHandlerSupport extends ChildServiceSupport implements ErrorHandler {
 
     protected final transient Logger log = LoggerFactory.getLogger(getClass());
 

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RoutePolicyProcessor.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RoutePolicyProcessor.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RoutePolicyProcessor.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/RoutePolicyProcessor.java Mon Aug 29 22:25:23 2011
@@ -23,8 +23,8 @@ import org.apache.camel.CamelContext;
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
 import org.apache.camel.Route;
+import org.apache.camel.StatefulService;
 import org.apache.camel.spi.RoutePolicy;
-import org.apache.camel.support.ServiceSupport;
 import org.apache.camel.support.SynchronizationAdapter;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -114,16 +114,16 @@ public class RoutePolicyProcessor extend
      * @return <tt>true</tt> to run
      */
     protected boolean isRoutePolicyRunAllowed(RoutePolicy policy) {
-        if (policy instanceof ServiceSupport) {
-            ServiceSupport ss = (ServiceSupport) policy;
+        if (policy instanceof StatefulService) {
+            StatefulService ss = (StatefulService) policy;
             return ss.isRunAllowed();
         }
         return true;
     }
 
     private static boolean isCamelStopping(CamelContext context) {
-        if (context instanceof ServiceSupport) {
-            ServiceSupport ss = (ServiceSupport) context;
+        if (context instanceof StatefulService) {
+            StatefulService ss = (StatefulService) context;
             return ss.isStopping() || ss.isStopped();
         }
         return false;

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/processor/aggregate/AggregateProcessor.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/aggregate/AggregateProcessor.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/processor/aggregate/AggregateProcessor.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/processor/aggregate/AggregateProcessor.java Mon Aug 29 22:25:23 2011
@@ -41,6 +41,7 @@ import org.apache.camel.Navigate;
 import org.apache.camel.NoSuchEndpointException;
 import org.apache.camel.Predicate;
 import org.apache.camel.Processor;
+import org.apache.camel.TimeoutMap;
 import org.apache.camel.impl.LoggingExceptionHandler;
 import org.apache.camel.processor.SendProcessor;
 import org.apache.camel.processor.Traceable;
@@ -48,15 +49,14 @@ import org.apache.camel.spi.AggregationR
 import org.apache.camel.spi.ExceptionHandler;
 import org.apache.camel.spi.RecoverableAggregationRepository;
 import org.apache.camel.spi.Synchronization;
+import org.apache.camel.support.DefaultTimeoutMap;
 import org.apache.camel.support.ServiceSupport;
-import org.apache.camel.util.DefaultTimeoutMap;
 import org.apache.camel.util.ExchangeHelper;
 import org.apache.camel.util.LRUCache;
 import org.apache.camel.util.ObjectHelper;
 import org.apache.camel.util.ServiceHelper;
 import org.apache.camel.util.StopWatch;
 import org.apache.camel.util.TimeUtils;
-import org.apache.camel.util.TimeoutMap;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/support/ChildServiceSupport.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/support/ChildServiceSupport.java?rev=1163044&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/support/ChildServiceSupport.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/support/ChildServiceSupport.java Mon Aug 29 22:25:23 2011
@@ -0,0 +1,137 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.camel.support;
+
+import java.util.LinkedHashSet;
+import java.util.Set;
+
+import org.apache.camel.util.ServiceHelper;
+
+public abstract class ChildServiceSupport extends ServiceSupport {
+    private Set<Object> childServices;
+    
+    public void start() throws Exception {
+        start(true);
+    }
+
+    public void start(boolean startChildren) throws Exception {
+        if (!started.get()) {
+            if (starting.compareAndSet(false, true)) {
+                boolean childrenStarted = false;
+                Exception ex = null;
+                try {
+                    if (childServices != null && startChildren) {
+                        ServiceHelper.startServices(childServices);
+                    }
+                    childrenStarted = true;
+                    doStart();
+                } catch (Exception e) {
+                    ex = e;
+                } finally {
+                    if (ex != null) {
+                        try {
+                            stop(childrenStarted);
+                        } catch (Exception e) {
+                            // Ignore exceptions as we want to show the original exception
+                        }
+                        throw ex;
+                    } else {
+                        started.set(true);
+                        starting.set(false);
+                        stopping.set(false);
+                        stopped.set(false);
+                        suspending.set(false);
+                        suspended.set(false);
+                        shutdown.set(false);
+                        shuttingdown.set(false);
+                    }
+                }
+            }
+        }
+    }
+    
+    private void stop(boolean childrenStarted) throws Exception {
+        if (stopping.compareAndSet(false, true)) {
+            try {
+                try {
+                    starting.set(false);
+                    suspending.set(false);
+                    if (childrenStarted) {
+                        doStop();
+                    }
+                } finally {
+                    started.set(false);
+                    suspended.set(false);
+                    if (childServices != null) {
+                        ServiceHelper.stopServices(childServices);
+                    }
+                }
+            } finally {
+                stopped.set(true);
+                stopping.set(false);
+                starting.set(false);
+                started.set(false);
+                suspending.set(false);
+                suspended.set(false);
+                shutdown.set(false);
+                shuttingdown.set(false);
+            }
+        }
+    }
+
+    public void stop() throws Exception {
+        if (!stopped.get()) {
+            stop(true);
+        }
+    }
+    
+    public void shutdown() throws Exception {
+        // ensure we are stopped first
+        stop();
+
+        if (shuttingdown.compareAndSet(false, true)) {
+            try {
+                try {
+                    doShutdown();
+                } finally {
+                    if (childServices != null) {
+                        ServiceHelper.stopAndShutdownServices(childServices);
+                    }
+                }
+            } finally {
+                // shutdown is also stopped so only set shutdown flags
+                shutdown.set(true);
+                shuttingdown.set(false);
+            }
+        }
+    }
+    
+    @SuppressWarnings("unchecked")
+    protected void addChildService(Object childService) {
+        synchronized (this) {
+            if (childServices == null) {
+                childServices = new LinkedHashSet();
+            }
+        }
+        childServices.add(childService);
+    }
+
+    protected boolean removeChildService(Object childService) {
+        return childServices != null && childServices.remove(childService);
+    }
+
+}

Copied: camel/trunk/camel-core/src/main/java/org/apache/camel/support/DefaultTimeoutMap.java (from r1161594, camel/trunk/camel-core/src/main/java/org/apache/camel/util/DefaultTimeoutMap.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/support/DefaultTimeoutMap.java?p2=camel/trunk/camel-core/src/main/java/org/apache/camel/support/DefaultTimeoutMap.java&p1=camel/trunk/camel-core/src/main/java/org/apache/camel/util/DefaultTimeoutMap.java&r1=1161594&r2=1163044&rev=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/util/DefaultTimeoutMap.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/support/DefaultTimeoutMap.java Mon Aug 29 22:25:23 2011
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.util;
+package org.apache.camel.support;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -29,7 +29,8 @@ import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 
-import org.apache.camel.support.ServiceSupport;
+import org.apache.camel.TimeoutMap;
+import org.apache.camel.util.ObjectHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/support/DefaultTimeoutMap.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/support/DefaultTimeoutMap.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/support/ServiceSupport.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/support/ServiceSupport.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/support/ServiceSupport.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/support/ServiceSupport.java Mon Aug 29 22:25:23 2011
@@ -17,15 +17,14 @@
 package org.apache.camel.support;
 
 import java.io.InputStream;
-import java.util.LinkedHashSet;
 import java.util.Properties;
-import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 
-import org.apache.camel.Service;
 import org.apache.camel.ServiceStatus;
 import org.apache.camel.ShutdownableService;
-import org.apache.camel.util.ServiceHelper;
+import org.apache.camel.StatefulService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * A useful base class which ensures that a service is only initialized once and
@@ -36,94 +35,68 @@ import org.apache.camel.util.ServiceHelp
  *
  * @version 
  */
-public abstract class ServiceSupport implements Service, ShutdownableService {
+public abstract class ServiceSupport implements ShutdownableService, StatefulService {
+    private static final transient Logger LOG = LoggerFactory.getLogger(ServiceSupport.class);
+
+    protected final AtomicBoolean started = new AtomicBoolean(false);
+    protected final AtomicBoolean starting = new AtomicBoolean(false);
+    protected final AtomicBoolean stopping = new AtomicBoolean(false);
+    protected final AtomicBoolean stopped = new AtomicBoolean(false);
+    protected final AtomicBoolean suspending = new AtomicBoolean(false);
+    protected final AtomicBoolean suspended = new AtomicBoolean(false);
+    protected final AtomicBoolean shuttingdown = new AtomicBoolean(false);
+    protected final AtomicBoolean shutdown = new AtomicBoolean(false);
 
-    private final AtomicBoolean started = new AtomicBoolean(false);
-    private final AtomicBoolean starting = new AtomicBoolean(false);
-    private final AtomicBoolean stopping = new AtomicBoolean(false);
-    private final AtomicBoolean stopped = new AtomicBoolean(false);
-    private final AtomicBoolean suspending = new AtomicBoolean(false);
-    private final AtomicBoolean suspended = new AtomicBoolean(false);
-    private final AtomicBoolean shuttingdown = new AtomicBoolean(false);
-    private final AtomicBoolean shutdown = new AtomicBoolean(false);
-    private Set<Object> childServices;
     private String version;
 
     public void start() throws Exception {
-        start(true);
-    }
-
-    public void start(boolean startChildren) throws Exception {
-        if (!started.get()) {
-            if (starting.compareAndSet(false, true)) {
-                boolean childrenStarted = false;
-                Exception ex = null;
-                try {
-                    if (childServices != null && startChildren) {
-                        ServiceHelper.startServices(childServices);
-                    }
-                    childrenStarted = true;
-                    doStart();
-                } catch (Exception e) {
-                    ex = e;
-                } finally {
-                    if (ex != null) {
-                        try {
-                            stop(childrenStarted);
-                        } catch (Exception e) {
-                            // Ignore exceptions as we want to show the original exception
-                        }
-                        throw ex;
-                    } else {
-                        started.set(true);
-                        starting.set(false);
-                        stopping.set(false);
-                        stopped.set(false);
-                        suspending.set(false);
-                        suspended.set(false);
-                        shutdown.set(false);
-                        shuttingdown.set(false);
-                    }
-                }
-            }
+        if (isStarting() || isStarted()) {
+            // only start service if not already started
+            LOG.trace("Service already started");
+            return;
+        }
+        if (starting.compareAndSet(false, true)) {
+            LOG.trace("Starting service");
+            doStart();
+            started.set(true);
+            starting.set(false);
+            stopping.set(false);
+            stopped.set(false);
+            suspending.set(false);
+            suspended.set(false);
+            shutdown.set(false);
+            shuttingdown.set(false);
         }
     }
     
-    private void stop(boolean childrenStarted) throws Exception {
-        if (stopping.compareAndSet(false, true)) {
-            try {
-                try {
-                    starting.set(false);
-                    suspending.set(false);
-                    if (childrenStarted) {
-                        doStop();
-                    }
-                } finally {
-                    started.set(false);
-                    suspended.set(false);
-                    if (childServices != null) {
-                        ServiceHelper.stopServices(childServices);
-                    }
-                }
-            } finally {
-                stopped.set(true);
-                stopping.set(false);
-                starting.set(false);
-                started.set(false);
-                suspending.set(false);
-                suspended.set(false);
-                shutdown.set(false);
-                shuttingdown.set(false);
-            }
-        }
-    }
-
     public void stop() throws Exception {
-        if (!stopped.get()) {
-            stop(true);
+        if (isStopped()) {
+            LOG.trace("Service already stopped");
+            return;
+        }
+        if (isStopping()) {
+            LOG.trace("Service already stopping");
+            return;
+        }
+        stopping.set(true);
+        try {
+            doStop();
+        } finally {
+            stopping.set(false);
+            stopped.set(true);
+            starting.set(false);
+            started.set(false);
+            suspending.set(false);
+            suspended.set(false);
+            shutdown.set(false);
+            shuttingdown.set(false);            
         }
     }
 
+    /* (non-Javadoc)
+     * @see org.apache.camel.support.StatefulService#suspend()
+     */
+    @Override
     public void suspend() throws Exception {
         if (!suspended.get()) {
             if (suspending.compareAndSet(false, true)) {
@@ -145,6 +118,10 @@ public abstract class ServiceSupport imp
         }
     }
 
+    /* (non-Javadoc)
+     * @see org.apache.camel.support.StatefulService#resume()
+     */
+    @Override
     public void resume() throws Exception {
         if (suspended.get()) {
             if (starting.compareAndSet(false, true)) {
@@ -164,19 +141,17 @@ public abstract class ServiceSupport imp
         }
     }
 
+    /* (non-Javadoc)
+     * @see org.apache.camel.support.StatefulService#shutdown()
+     */
+    @Override
     public void shutdown() throws Exception {
         // ensure we are stopped first
         stop();
 
         if (shuttingdown.compareAndSet(false, true)) {
             try {
-                try {
-                    doShutdown();
-                } finally {
-                    if (childServices != null) {
-                        ServiceHelper.stopAndShutdownService(childServices);
-                    }
-                }
+                doShutdown();
             } finally {
                 // shutdown is also stopped so only set shutdown flags
                 shutdown.set(true);
@@ -185,9 +160,10 @@ public abstract class ServiceSupport imp
         }
     }
 
-    /**
-     * Returns the current status
+    /* (non-Javadoc)
+     * @see org.apache.camel.support.StatefulService#getStatus()
      */
+    @Override
     public ServiceStatus getStatus() {
         // lets check these in oldest first as these flags can be changing in a concurrent world
         if (isStarting()) {
@@ -213,54 +189,58 @@ public abstract class ServiceSupport imp
         return ServiceStatus.Stopped;
     }
     
-    /**
-     * @return true if this service has been started
+    /* (non-Javadoc)
+     * @see org.apache.camel.support.StatefulService#isStarted()
      */
+    @Override
     public boolean isStarted() {
         return started.get();
     }
 
-    /**
-     * @return true if this service is being started
+    /* (non-Javadoc)
+     * @see org.apache.camel.support.StatefulService#isStarting()
      */
+    @Override
     public boolean isStarting() {
         return starting.get();
     }
 
-    /**
-     * @return true if this service is in the process of stopping
+    /* (non-Javadoc)
+     * @see org.apache.camel.support.StatefulService#isStopping()
      */
+    @Override
     public boolean isStopping() {
         return stopping.get();
     }
 
-    /**
-     * @return true if this service is stopped
+    /* (non-Javadoc)
+     * @see org.apache.camel.support.StatefulService#isStopped()
      */
+    @Override
     public boolean isStopped() {
         return stopped.get();
     }
 
-    /**
-     * @return true if this service is in the process of suspending
+    /* (non-Javadoc)
+     * @see org.apache.camel.support.StatefulService#isSuspending()
      */
+    @Override
     public boolean isSuspending() {
         return suspending.get();
     }
 
-    /**
-     * @return true if this service is suspended
+    /* (non-Javadoc)
+     * @see org.apache.camel.support.StatefulService#isSuspended()
      */
+    @Override
     public boolean isSuspended() {
         return suspended.get();
     }
 
-    /**
-     * Helper methods so the service knows if it should keep running.
-     * Returns <tt>false</tt> if the service is being stopped or is stopped.
-     *
-     * @return <tt>true</tt> if the service should continue to run.
+    /* (non-Javadoc)
+     * @see org.apache.camel.support.StatefulService#isRunAllowed()
      */
+    @Override
     public boolean isRunAllowed() {
         return !(stopping.get() || stopped.get());
     }
@@ -288,23 +268,10 @@ public abstract class ServiceSupport imp
         // noop
     }
 
-    @SuppressWarnings("unchecked")
-    protected void addChildService(Object childService) {
-        synchronized (this) {
-            if (childServices == null) {
-                childServices = new LinkedHashSet();
-            }
-        }
-        childServices.add(childService);
-    }
-
-    protected boolean removeChildService(Object childService) {
-        return childServices != null && childServices.remove(childService);
-    }
-
-    /**
-     * Returns the version of this service
+    /* (non-Javadoc)
+     * @see org.apache.camel.support.StatefulService#getVersion()
      */
+    @Override
     public synchronized String getVersion() {
         if (version != null) {
             return version;

Copied: camel/trunk/camel-core/src/main/java/org/apache/camel/support/TimeoutMapEntry.java (from r1161594, camel/trunk/camel-core/src/main/java/org/apache/camel/util/TimeoutMapEntry.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/support/TimeoutMapEntry.java?p2=camel/trunk/camel-core/src/main/java/org/apache/camel/support/TimeoutMapEntry.java&p1=camel/trunk/camel-core/src/main/java/org/apache/camel/util/TimeoutMapEntry.java&r1=1161594&r2=1163044&rev=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/util/TimeoutMapEntry.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/support/TimeoutMapEntry.java Mon Aug 29 22:25:23 2011
@@ -14,10 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.util;
+package org.apache.camel.support;
 
 import java.util.Map;
 
+import org.apache.camel.TimeoutMap;
+
 /**
  * Represents an entry in a {@link TimeoutMap}
  *

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/support/TimeoutMapEntry.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: camel/trunk/camel-core/src/main/java/org/apache/camel/support/TimeoutMapEntry.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/util/EventHelper.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/util/EventHelper.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/util/EventHelper.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/util/EventHelper.java Mon Aug 29 22:25:23 2011
@@ -24,9 +24,9 @@ import org.apache.camel.Endpoint;
 import org.apache.camel.Exchange;
 import org.apache.camel.Processor;
 import org.apache.camel.Route;
+import org.apache.camel.StatefulService;
 import org.apache.camel.spi.EventFactory;
 import org.apache.camel.spi.EventNotifier;
-import org.apache.camel.support.ServiceSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -559,8 +559,8 @@ public final class EventHelper {
     private static void doNotifyEvent(EventNotifier notifier, EventObject event) {
         // only notify if notifier is started
         boolean started = true;
-        if (notifier instanceof ServiceSupport) {
-            started = ((ServiceSupport) notifier).isStarted();
+        if (notifier instanceof StatefulService) {
+            started = ((StatefulService) notifier).isStarted();
         }
         if (!started) {
             LOG.debug("Ignoring notifying event {}. The EventNotifier has not been started yet: {}", event, notifier);

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/util/ServiceHelper.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/util/ServiceHelper.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/util/ServiceHelper.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/util/ServiceHelper.java Mon Aug 29 22:25:23 2011
@@ -22,8 +22,8 @@ import java.util.List;
 
 import org.apache.camel.Service;
 import org.apache.camel.ShutdownableService;
+import org.apache.camel.StatefulService;
 import org.apache.camel.SuspendableService;
-import org.apache.camel.support.ServiceSupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -45,19 +45,19 @@ public final class ServiceHelper {
      * Starts all of the given services
      */
     public static void startService(Object value) throws Exception {
-        if (isStarted(value)) {
-            // only start service if not already started
-            LOG.trace("Service already started: {}", value);
-            return;
-        }
         if (value instanceof Service) {
-            Service service = (Service)value;
-            LOG.trace("Starting service: {}", service);
-            service.start();
+            startService((Service)value);
         } else if (value instanceof Collection) {
             startServices((Collection<?>)value);
         }
     }
+    
+    /**
+     * Starts all of the given services
+     */
+    public static void startService(Service service) throws Exception {
+        service.start();
+    }
 
     /**
      * Starts all of the given services
@@ -151,19 +151,14 @@ public final class ServiceHelper {
     /**
      * Stops and shutdowns all of the given services, throwing the first exception caught
      */
-    public static void stopAndShutdownService(Object value) throws Exception {
-        if (value instanceof Service) {
-            // must stop it first
-            stopService(value);
-        }
+    public static void stopAndShutdownService(Service value) throws Exception {
+        stopService(value);
 
         // then try to shutdown
         if (value instanceof ShutdownableService) {
             ShutdownableService service = (ShutdownableService)value;
             LOG.trace("Shutting down service {}", value);
             service.shutdown();
-        } else if (value instanceof Collection) {
-            stopAndShutdownServices((Collection<?>)value);
         }
     }
 
@@ -243,7 +238,7 @@ public final class ServiceHelper {
      * <tt>false</tt> if the service is already in the desired state.
      * @throws Exception is thrown if error occurred
      */
-    public static boolean resumeService(Object service) throws Exception {
+    public static boolean resumeService(Service service) throws Exception {
         if (service instanceof SuspendableService) {
             SuspendableService ss = (SuspendableService) service;
             if (ss.isSuspended()) {
@@ -322,8 +317,8 @@ public final class ServiceHelper {
      * @return <tt>true</tt> if already stopped, otherwise <tt>false</tt>
      */
     public static boolean isStopped(Object value) {
-        if (value instanceof ServiceSupport) {
-            ServiceSupport service = (ServiceSupport) value;
+        if (value instanceof StatefulService) {
+            StatefulService service = (StatefulService) value;
             if (service.isStopping() || service.isStopped()) {
                 return true;
             }
@@ -337,8 +332,8 @@ public final class ServiceHelper {
      * @return <tt>true</tt> if already started, otherwise <tt>false</tt>
      */
     public static boolean isStarted(Object value) {
-        if (value instanceof ServiceSupport) {
-            ServiceSupport service = (ServiceSupport) value;
+        if (value instanceof StatefulService) {
+            StatefulService service = (StatefulService) value;
             if (service.isStarting() || service.isStarted()) {
                 return true;
             }

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/processor/Endpoint2MustBeStartedBeforeSendProcessorTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/processor/Endpoint2MustBeStartedBeforeSendProcessorTest.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/processor/Endpoint2MustBeStartedBeforeSendProcessorTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/processor/Endpoint2MustBeStartedBeforeSendProcessorTest.java Mon Aug 29 22:25:23 2011
@@ -107,8 +107,7 @@ public class Endpoint2MustBeStartedBefor
 
         order = "";
         context.stop();
-        // will invoke StopEndpoint twice as shutdown will ensure we are stopped first
-        assertEquals("StopConsumerStopProducerStopEndpointStopEndpoint", order);
+        assertEquals("StopConsumerStopProducerStopEndpoint", order);
     }
 
     @Override
@@ -141,14 +140,12 @@ public class Endpoint2MustBeStartedBefor
         // this is however discouraged, as you should prefer to use doStart/doStop
 
         @Override
-        public void start() throws Exception {
-            super.start();
+        public void doStart() throws Exception {
             order += "Endpoint";
         }
 
         @Override
-        public void stop() throws Exception {
-            super.stop();
+        public void doStop() throws Exception {
             order += "StopEndpoint";
         }
     }

Copied: camel/trunk/camel-core/src/test/java/org/apache/camel/support/DefaultTimeoutMapTest.java (from r1161594, camel/trunk/camel-core/src/test/java/org/apache/camel/util/DefaultTimeoutMapTest.java)
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/support/DefaultTimeoutMapTest.java?p2=camel/trunk/camel-core/src/test/java/org/apache/camel/support/DefaultTimeoutMapTest.java&p1=camel/trunk/camel-core/src/test/java/org/apache/camel/util/DefaultTimeoutMapTest.java&r1=1161594&r2=1163044&rev=1163044&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/util/DefaultTimeoutMapTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/support/DefaultTimeoutMapTest.java Mon Aug 29 22:25:23 2011
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.camel.util;
+package org.apache.camel.support;
 
 import java.util.ArrayList;
 import java.util.List;

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

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

Modified: camel/trunk/components/camel-jms/src/main/java/org/apache/camel/component/jms/reply/CorrelationMap.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-jms/src/main/java/org/apache/camel/component/jms/reply/CorrelationMap.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/components/camel-jms/src/main/java/org/apache/camel/component/jms/reply/CorrelationMap.java (original)
+++ camel/trunk/components/camel-jms/src/main/java/org/apache/camel/component/jms/reply/CorrelationMap.java Mon Aug 29 22:25:23 2011
@@ -18,7 +18,7 @@ package org.apache.camel.component.jms.r
 
 import java.util.concurrent.ScheduledExecutorService;
 
-import org.apache.camel.util.DefaultTimeoutMap;
+import org.apache.camel.support.DefaultTimeoutMap;
 
 /**
  * @version 

Modified: camel/trunk/components/camel-quickfix/src/test/java/org/apache/camel/component/quickfixj/QuickfixjComponentTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-quickfix/src/test/java/org/apache/camel/component/quickfixj/QuickfixjComponentTest.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/components/camel-quickfix/src/test/java/org/apache/camel/component/quickfixj/QuickfixjComponentTest.java (original)
+++ camel/trunk/components/camel-quickfix/src/test/java/org/apache/camel/component/quickfixj/QuickfixjComponentTest.java Mon Aug 29 22:25:23 2011
@@ -33,10 +33,10 @@ import org.apache.camel.ExchangePattern;
 import org.apache.camel.MultipleConsumersSupport;
 import org.apache.camel.Processor;
 import org.apache.camel.Producer;
+import org.apache.camel.StatefulService;
 import org.apache.camel.component.quickfixj.converter.QuickfixjConverters;
 import org.apache.camel.impl.DefaultCamelContext;
 import org.apache.camel.impl.converter.StaticMethodTypeConverter;
-import org.apache.camel.support.ServiceSupport;
 import org.apache.camel.util.ServiceHelper;
 import org.apache.mina.common.TransportType;
 import org.junit.After;
@@ -228,7 +228,7 @@ public class QuickfixjComponentTest {
         ServiceHelper.startService(consumer);
 
         // Endpoint automatically starts the consumer
-        assertThat(((ServiceSupport)consumer).isStarted(), is(true));
+        assertThat(((StatefulService)consumer).isStarted(), is(true));
         
         component.start();
         

Modified: camel/trunk/components/camel-quickfix/src/test/java/org/apache/camel/component/quickfixj/QuickfixjConsumerTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/components/camel-quickfix/src/test/java/org/apache/camel/component/quickfixj/QuickfixjConsumerTest.java?rev=1163044&r1=1163043&r2=1163044&view=diff
==============================================================================
--- camel/trunk/components/camel-quickfix/src/test/java/org/apache/camel/component/quickfixj/QuickfixjConsumerTest.java (original)
+++ camel/trunk/components/camel-quickfix/src/test/java/org/apache/camel/component/quickfixj/QuickfixjConsumerTest.java Mon Aug 29 22:25:23 2011
@@ -20,7 +20,7 @@ import org.apache.camel.Endpoint;
 import org.apache.camel.Exchange;
 import org.apache.camel.ExchangePattern;
 import org.apache.camel.Processor;
-import org.apache.camel.support.ServiceSupport;
+import org.apache.camel.StatefulService;
 import org.hamcrest.CoreMatchers;
 import org.junit.Assert;
 import org.junit.Before;
@@ -67,7 +67,7 @@ public class QuickfixjConsumerTest {
         QuickfixjConsumer consumer = new QuickfixjConsumer(mockEndpoint, mockProcessor);
         
         Assert.assertThat("Consumer should not be automatically started", 
-            ((ServiceSupport)consumer).isStarted(), CoreMatchers.is(false));
+            ((StatefulService)consumer).isStarted(), CoreMatchers.is(false));
         
         consumer.onExchange(mockExchange);
         
@@ -75,7 +75,7 @@ public class QuickfixjConsumerTest {
         Mockito.verifyZeroInteractions(mockProcessor);
         
         consumer.start();
-        Assert.assertThat(((ServiceSupport)consumer).isStarted(), CoreMatchers.is(true));
+        Assert.assertThat(((StatefulService)consumer).isStarted(), CoreMatchers.is(true));
         
         consumer.onExchange(mockExchange);