You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by js...@apache.org on 2007/09/03 18:13:24 UTC

svn commit: r572379 - in /activemq/camel/trunk/camel-core/src: main/java/org/apache/camel/impl/ main/java/org/apache/camel/management/ main/java/org/apache/camel/spi/ test/java/org/apache/camel/ test/java/org/apache/camel/management/

Author: jstrachan
Date: Mon Sep  3 09:13:20 2007
New Revision: 572379

URL: http://svn.apache.org/viewvc?rev=572379&view=rev
Log:
applied the great patch from Hadrian Zbarcea for CAMEL-10 to add optional interceptors to publish statistics MBeans

Added:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/Counter.java   (with props)
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationProcessor.java   (with props)
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/PerformanceCounter.java   (with props)
Modified:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultLifecycleStrategy.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/CamelNamingStrategy.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/ManagedRoute.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/LifecycleStrategy.java
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/TestSupport.java
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationUsingDefaultsTest.java

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java?rev=572379&r1=572378&r2=572379&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultCamelContext.java Mon Sep  3 09:13:20 2007
@@ -424,6 +424,7 @@
     protected void startRouteDefinitions(Collection<RouteType> list) throws Exception {
         if (list != null) {
             for (RouteType route : list) {
+            	lifecycleStrategy.beforeStartRouteType(this, route);
                 route.addRoutes(this);
             }
         }

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultLifecycleStrategy.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultLifecycleStrategy.java?rev=572379&r1=572378&r2=572379&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultLifecycleStrategy.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/impl/DefaultLifecycleStrategy.java Mon Sep  3 09:13:20 2007
@@ -20,6 +20,7 @@
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.Endpoint;
+import org.apache.camel.model.RouteType;
 import org.apache.camel.spi.LifecycleStrategy;
 import org.apache.camel.Route;
 import org.apache.camel.Service;
@@ -39,6 +40,10 @@
 	}
 
 	public void onRoutesAdd(Collection<Route> routes) {
+		// do nothing
+	}
+
+	public void beforeStartRouteType(CamelContext context, RouteType routeType) {
 		// do nothing
 	}
 }

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/CamelNamingStrategy.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/CamelNamingStrategy.java?rev=572379&r1=572378&r2=572379&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/CamelNamingStrategy.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/CamelNamingStrategy.java Mon Sep  3 09:13:20 2007
@@ -19,7 +19,6 @@
 
 import java.net.InetAddress;
 import java.net.UnknownHostException;
-import java.util.Hashtable;
 
 import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
@@ -31,13 +30,13 @@
 
 	public static final String VALUE_UNKNOWN = "unknown";
 	public static final String KEY_CONTEXT = "context";
-	public static final String KEY_ENDPOINT = "endpoint";
-	public static final String KEY_ROUTE = "route";
+	public static final String KEY_CLASS = "class";
 	public static final String KEY_TYPE = "type";
 	public static final String KEY_NAME = "name";
-	public static final String TYPE_ENDPOINTS = "Endpoints";
-	public static final String TYPE_SERVICES = "Services";
-	public static final String TYPE_ROUTES = "Routes";
+	public static final String KEY_ROUTE = "route";
+	public static final String CLASS_ENDPOINTS = "endpoints";
+	public static final String CLASS_SERVICES = "services";
+	public static final String CLASS_ROUTES = "routes";
 	
 	protected String domainName = "org.apache.camel";
 	protected String hostName = "locahost";
@@ -50,7 +49,7 @@
 			hostName = InetAddress.getLocalHost().getHostName();
 		}
 		catch (UnknownHostException ex) {
-			// ignore
+			// ignore, use the default "locahost"
 		}
 	}
 
@@ -64,10 +63,11 @@
 	 * @throws MalformedObjectNameException
 	 */
 	public ObjectName getObjectName(CamelContext context) throws MalformedObjectNameException {
-		Hashtable<String, String> keys = new Hashtable<String, String>();
-		keys.put(KEY_CONTEXT, getContextId(context));
-		keys.put(KEY_NAME, "camel");
-		return new ObjectName(domainName, keys);
+		StringBuffer buffer = new StringBuffer();
+		buffer.append(domainName + ":");
+		buffer.append(KEY_CONTEXT + "=" + getContextId(context) + ",");
+		buffer.append(KEY_NAME + "=" + "context");
+		return new ObjectName(buffer.toString());
 	}
 
 	/**
@@ -81,11 +81,13 @@
 	 */
 	public ObjectName getObjectName(ManagedEndpoint mbean) throws MalformedObjectNameException {
 		Endpoint ep = mbean.getEndpoint();
-		Hashtable<String, String> keys = new Hashtable<String, String>();
-		keys.put(KEY_CONTEXT, getContextId(ep.getContext()));
-		keys.put(KEY_TYPE, TYPE_ENDPOINTS);
-		keys.put(KEY_ENDPOINT, getEndpointId(ep));
-		return new ObjectName(domainName, keys);
+		
+		StringBuffer buffer = new StringBuffer();
+		buffer.append(domainName + ":");
+		buffer.append(KEY_CONTEXT + "=" + getContextId(ep.getContext()) + ",");
+		buffer.append(KEY_CLASS + "=" + CLASS_ENDPOINTS + ",");
+		buffer.append(KEY_NAME + "=" + getEndpointId(ep));
+		return new ObjectName(buffer.toString());
 	}
 
 	/**
@@ -98,11 +100,12 @@
 	 * @throws MalformedObjectNameException
 	 */
 	public ObjectName getObjectName(CamelContext context, ManagedService mbean) throws MalformedObjectNameException {
-		Hashtable<String, String> keys = new Hashtable<String, String>();
-		keys.put(KEY_CONTEXT, getContextId(context));
-		keys.put(KEY_TYPE, TYPE_SERVICES);
-		keys.put(KEY_ENDPOINT, Integer.toHexString(mbean.getService().hashCode()));
-		return new ObjectName(domainName, keys);
+		StringBuffer buffer = new StringBuffer();
+		buffer.append(domainName + ":");
+		buffer.append(KEY_CONTEXT + "=" + getContextId(context) + ",");
+		buffer.append(KEY_CLASS + "=" + CLASS_SERVICES + ",");
+		buffer.append(KEY_NAME + "=" + Integer.toHexString(mbean.getService().hashCode()));
+		return new ObjectName(buffer.toString());
 	}
 
 	/**
@@ -115,13 +118,34 @@
 	 * @throws MalformedObjectNameException
 	 */
 	public ObjectName getObjectName(ManagedRoute mbean) throws MalformedObjectNameException {
-		Hashtable<String, String> keys = new Hashtable<String, String>();
 		Endpoint ep = mbean.getRoute().getEndpoint();
 		String ctxid = ep != null ? getContextId(ep.getContext()) : VALUE_UNKNOWN;
-		keys.put(KEY_CONTEXT, ctxid);
-		keys.put(KEY_TYPE, TYPE_ROUTES);
-		keys.put(KEY_ENDPOINT, getEndpointId(ep));
-		return new ObjectName(domainName, keys);
+		
+		StringBuffer buffer = new StringBuffer();
+		buffer.append(domainName + ":");
+		buffer.append(KEY_CONTEXT + "=" + ctxid + ",");
+		buffer.append(KEY_CLASS + "=" + CLASS_ROUTES + ",");
+		buffer.append(KEY_ROUTE + "=" + getEndpointId(ep));
+		return new ObjectName(buffer.toString());
+	}
+	
+	/**
+	 * Implements the naming strategy for a {@see PerformanceCounter}.
+	 * The convention used for a {@see ManagedEndpoint} ObjectName is
+	 * "<domain>:context=<context>,type=Routes,endpoint=[urlPrefix]localPart".
+	 * 
+	 * @param mbean
+	 * @return generated ObjectName
+	 * @throws MalformedObjectNameException
+	 */
+	public ObjectName getObjectName(CamelContext context, PerformanceCounter mbean) throws MalformedObjectNameException {
+		StringBuffer buffer = new StringBuffer();
+		buffer.append(domainName + ":");
+		buffer.append(KEY_CONTEXT + "=" + getContextId(context) + ",");
+		buffer.append(KEY_CLASS + "=" + CLASS_ROUTES + ",");
+		buffer.append(KEY_ROUTE + "=" + "Route.Counter" + ",");     // TODO: figure out the route id
+		buffer.append(KEY_NAME + "=" + "Stats");
+		return new ObjectName(buffer.toString());
 	}
 	
 	protected String getContextId(CamelContext context) {

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/Counter.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/Counter.java?rev=572379&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/Counter.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/Counter.java Mon Sep  3 09:13:20 2007
@@ -0,0 +1,46 @@
+/**
+ *
+ * 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.concurrent.atomic.AtomicLong;
+
+import org.springframework.jmx.export.annotation.ManagedAttribute;
+import org.springframework.jmx.export.annotation.ManagedOperation;
+import org.springframework.jmx.export.annotation.ManagedResource;
+
+@ManagedResource(
+        description="Counter", 
+        currencyTimeLimit=15)
+public class Counter {
+
+	protected AtomicLong numExchanges = new AtomicLong(0L);
+	
+	@ManagedOperation(description = "Reset counters")
+	public void reset() {
+		numExchanges.set(0L);
+	}
+	
+	@ManagedAttribute(description = "Total number of exchanges")
+	public long getNumExchanges() throws Exception {
+		return numExchanges.get();
+	}
+
+	public long increment() {
+		return numExchanges.incrementAndGet();
+	}
+}

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

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java?rev=572379&r1=572378&r2=572379&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java Mon Sep  3 09:13:20 2007
@@ -29,9 +29,14 @@
 import org.apache.camel.Service;
 import org.apache.camel.impl.DefaultCamelContext;
 import org.apache.camel.impl.ServiceSupport;
+import org.apache.camel.model.RouteType;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 
 public class InstrumentationLifecycleStrategy implements LifecycleStrategy {
-	InstrumentationAgent agent;
+    private static final transient Log LOG = LogFactory.getLog(InstrumentationProcessor.class);
+
+    InstrumentationAgent agent;
 	CamelNamingStrategy naming;
 	
 	public InstrumentationLifecycleStrategy(InstrumentationAgent agent) {
@@ -47,7 +52,7 @@
 				agent.register(ms, naming.getObjectName(dc));
 			}
 			catch(JMException e) {
-				// log a WARN
+				LOG.warn("Could not register CamelContext MBean", e);
 			}
 		}
 	}
@@ -58,7 +63,7 @@
 			agent.register(me, naming.getObjectName(me));
 		}
 		catch(JMException e) {
-			// log a WARN
+			LOG.warn("Could not register Endpoint MBean", e);
 		}
 	}
 
@@ -69,7 +74,7 @@
 				agent.register(mr, naming.getObjectName(mr));
 			}
 			catch(JMException e) {
-				// log a WARN
+				LOG.warn("Could not register Route MBean", e);
 			}
 		}
 	}
@@ -81,8 +86,20 @@
 				agent.register(ms, naming.getObjectName(context, ms));
 			}
 			catch(JMException e) {
-				// log a WARN
+				LOG.warn("Could not register Service MBean", e);
 			}
+		}
+	}
+
+	public void beforeStartRouteType(CamelContext context, RouteType routeType) {
+		PerformanceCounter mc = new PerformanceCounter();
+		routeType.intercept(new InstrumentationProcessor(mc));
+
+		try {
+			agent.register(mc, naming.getObjectName(context, mc));
+		}
+		catch(JMException e) {
+			LOG.warn("Could not register Counter MBean", e);
 		}
 	}
 }

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationProcessor.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationProcessor.java?rev=572379&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationProcessor.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationProcessor.java Mon Sep  3 09:13:20 2007
@@ -0,0 +1,43 @@
+/**
+ *
+ * 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 org.apache.camel.Exchange;
+import org.apache.camel.processor.DelegateProcessor;
+
+public class InstrumentationProcessor extends DelegateProcessor {
+
+    private PerformanceCounter counter;
+    
+    InstrumentationProcessor(PerformanceCounter counter) {
+    	this.counter = counter;
+    }
+    
+    public void process(Exchange exchange) throws Exception {
+    	long startTime = System.nanoTime();
+        super.process(exchange);
+        if (counter != null) {
+            if (exchange.getException() == null) {
+            	counter.completedExchange((System.nanoTime() - startTime) / 1000);
+            }
+            else {
+            	counter.completedExchange();
+            }
+        }
+    }
+}

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

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/ManagedRoute.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/ManagedRoute.java?rev=572379&r1=572378&r2=572379&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/ManagedRoute.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/ManagedRoute.java Mon Sep  3 09:13:20 2007
@@ -32,9 +32,11 @@
 	
 	public static final String VALUE_UNKNOWN = "Unknown";
 	private Route route;
+	private String description;
 	
 	ManagedRoute(Route route) {
 		this.route = route;
+		this.description = route.toString();
 	}
 
 	public Route getRoute() {
@@ -47,11 +49,16 @@
 		return ep != null ? ep.getEndpointUri() : VALUE_UNKNOWN;
 	}
 	
+	@ManagedAttribute(description = "Route description")
+	public String getDescription() {
+		return description;
+	}
+
 	@ManagedOperation(description = "Start Route")
     public void start() throws IOException {
 		throw new IOException("Not supported");
     }
-	
+
 	@ManagedOperation(description = "Stop Route")
     public void stop() throws IOException {
 		throw new IOException("Not supported");

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/PerformanceCounter.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/PerformanceCounter.java?rev=572379&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/PerformanceCounter.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/management/PerformanceCounter.java Mon Sep  3 09:13:20 2007
@@ -0,0 +1,87 @@
+/**
+ *
+ * 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.concurrent.atomic.AtomicLong;
+
+import org.springframework.jmx.export.annotation.ManagedAttribute;
+import org.springframework.jmx.export.annotation.ManagedOperation;
+import org.springframework.jmx.export.annotation.ManagedResource;
+
+@ManagedResource(
+        description="PerformanceCounter", 
+        currencyTimeLimit=15)
+public class PerformanceCounter extends Counter {
+
+	private AtomicLong numCompleted = new AtomicLong(0L);
+	private long minProcessingTime = -1L;
+	private long maxProcessingTime = -1L;
+	private double totalProcessingTime = 0;
+	
+	@Override
+	@ManagedOperation(description = "Reset counters")
+	public synchronized void reset() {
+		super.reset();
+		numCompleted.set(0L);
+		minProcessingTime = 0L;
+		maxProcessingTime = 0L;
+		totalProcessingTime = 0;
+	}
+	
+	@ManagedAttribute(description = "Number of successful exchanges")
+	public long getNumCompleted() throws Exception {
+		return numCompleted.get();
+	}
+
+	@ManagedAttribute(description = "Number of failed exchanges")
+	public long getNumFailed() throws Exception {
+		return numExchanges.get() - numCompleted.get();
+	}
+
+	@ManagedAttribute(description = "Min Processing Time [usec]")
+	public synchronized long getMinProcessingTime() throws Exception {
+		return minProcessingTime;
+	}
+
+	@ManagedAttribute(description = "Mean Processing Time [usec]")
+	public synchronized long getMeanProcessingTime() throws Exception {
+		long count = numCompleted.get();
+		return count > 0 ? (long)totalProcessingTime / count : 0L;
+	}
+
+	@ManagedAttribute(description = "Max Processing Time [usec]")
+	public synchronized long getMaxProcessingTime() throws Exception {
+		return maxProcessingTime;
+	}
+	
+	public synchronized void completedExchange(long time) {
+		increment();
+		numCompleted.incrementAndGet();
+		totalProcessingTime += time;
+		if (minProcessingTime < 0 || time < minProcessingTime) {
+			minProcessingTime = time;
+		}
+		if (time > maxProcessingTime) {
+			maxProcessingTime = time;
+		}
+	}
+
+	public void completedExchange() {
+		numExchanges.incrementAndGet();
+	}
+}

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

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/LifecycleStrategy.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/LifecycleStrategy.java?rev=572379&r1=572378&r2=572379&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/LifecycleStrategy.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/LifecycleStrategy.java Mon Sep  3 09:13:20 2007
@@ -22,6 +22,7 @@
 import org.apache.camel.Endpoint;
 import org.apache.camel.Route;
 import org.apache.camel.Service;
+import org.apache.camel.model.RouteType;
 
 public interface LifecycleStrategy {
 
@@ -36,7 +37,7 @@
 	void onEndpointAdd(Endpoint endpoint);
 
 	/**
-     * Notification on adding a {@see Route}.
+     * Notification on adding a {@see Service}.
      */
 	void onServiceAdd(CamelContext context, Service service);
 	
@@ -44,4 +45,10 @@
      * Notification on adding {@see Route}(s).
      */
 	void onRoutesAdd(Collection<Route> routes);
+	
+	/**
+     * Notification on adding {@see Route}(s).
+	 * @param context TODO
+     */
+	void beforeStartRouteType(CamelContext context, RouteType routeType);
 }

Modified: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/TestSupport.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/TestSupport.java?rev=572379&r1=572378&r2=572379&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/TestSupport.java (original)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/TestSupport.java Mon Sep  3 09:13:20 2007
@@ -45,7 +45,7 @@
      * Returns a value builder for the given header
      */
 
-    public ValueBuilder header(String name) {
+    public <E extends Exchange> ValueBuilder<E> header(String name) {
         return Builder.header(name);
     }
 

Modified: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationUsingDefaultsTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationUsingDefaultsTest.java?rev=572379&r1=572378&r2=572379&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationUsingDefaultsTest.java (original)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationUsingDefaultsTest.java Mon Sep  3 09:13:20 2007
@@ -18,8 +18,12 @@
 package org.apache.camel.management;
 
 import java.io.IOException;
+import java.util.Iterator;
 import java.util.Set;
 
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanInfo;
+import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
 import org.apache.camel.CamelContext;
@@ -42,7 +46,9 @@
 	
     protected CamelContext createCamelContext() throws Exception {
     	CamelContext context = super.createCamelContext();
+    	
     	createInstrumentationAgent(context, DEFAULT_PORT);
+
     	return context;
     }
 
@@ -67,10 +73,52 @@
     }
     
     public void testMBeansRegistered() throws Exception {
-        resolveMandatoryEndpoint("mock:end", MockEndpoint.class);
-        
-        ObjectName name = new ObjectName(domainName + ":type=Endpoints,*");
+    	resolveMandatoryEndpoint("mock:end", MockEndpoint.class);
+
+        ObjectName name = new ObjectName(domainName + ":class=endpoints,*");
         Set s = iAgent.getMBeanServer().queryNames(name, null);
         assertTrue(s.size() == 2);
+    }
+    
+    public void testCounters() throws Exception {
+    	MockEndpoint resultEndpoint = resolveMandatoryEndpoint("mock:end", MockEndpoint.class);
+        resultEndpoint.expectedBodiesReceived("<hello>world!</hello>");
+        sendBody("direct:start", "<hello>world!</hello>");
+
+        resultEndpoint.assertIsSatisfied();
+        
+        MBeanServer mbs = iAgent.getMBeanServer(); 
+        ObjectName name = new ObjectName(domainName + ":name=Stats,*");
+        Set s = mbs.queryNames(name, null);
+        assertTrue(s.size() == 1);
+        
+        Iterator iter = s.iterator();
+        ObjectName pcob = (ObjectName)iter.next();
+        
+        Long valueofNumExchanges = (Long)mbs.getAttribute(pcob, "NumExchanges");
+        assertNotNull("Expected attribute not found. MBean registerred under a " + 
+        		"'<domain>:name=Stats,*' key must be of type PerformanceCounter.class", valueofNumExchanges); 
+        assertTrue(valueofNumExchanges == 1); 
+        Long valueofNumCompleted = (Long)mbs.getAttribute(pcob, "NumCompleted");
+        assertNotNull("Expected attribute not found. MBean registerred under a " + 
+        		"'<domain>:name=Stats,*' key must be of type PerformanceCounter.class", valueofNumCompleted); 
+        assertTrue(valueofNumCompleted == 1); 
+        Long valueofNumFailed = (Long)mbs.getAttribute(pcob, "NumFailed");
+        assertNotNull("Expected attribute not found. MBean registerred under a " + 
+        		"'<domain>:name=Stats,*' key must be of type PerformanceCounter.class", valueofNumFailed); 
+        assertTrue(valueofNumFailed == 0); 
+        Long valueofMinProcessingTime = (Long)mbs.getAttribute(pcob, "MinProcessingTime");
+        assertNotNull("Expected attribute not found. MBean registerred under a " + 
+        		"'<domain>:name=Stats,*' key must be of type PerformanceCounter.class", valueofMinProcessingTime); 
+        assertTrue(valueofMinProcessingTime > 0); 
+        Long valueofMaxProcessingTime = (Long)mbs.getAttribute(pcob, "MaxProcessingTime");
+        assertNotNull("Expected attribute not found. MBean registerred under a " + 
+        		"'<domain>:name=Stats,*' key must be of type PerformanceCounter.class", valueofMaxProcessingTime); 
+        assertTrue(valueofMaxProcessingTime > 0); 
+        Long valueofMeanProcessingTime = (Long)mbs.getAttribute(pcob, "MeanProcessingTime");
+        assertNotNull("Expected attribute not found. MBean registerred under a " + 
+        		"'<domain>:name=Stats,*' key must be of type PerformanceCounter.class", valueofMeanProcessingTime); 
+        assertTrue(valueofMeanProcessingTime >= valueofMinProcessingTime && 
+        		valueofMeanProcessingTime <= valueofMaxProcessingTime); 
     }
 }