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/09/01 09:51:21 UTC

svn commit: r809872 - in /camel/trunk/camel-core/src: main/java/org/apache/camel/component/timer/ main/java/org/apache/camel/management/ main/java/org/apache/camel/management/mbean/ main/java/org/apache/camel/spi/ test/java/org/apache/camel/management/

Author: davsclaus
Date: Tue Sep  1 07:51:20 2009
New Revision: 809872

URL: http://svn.apache.org/viewvc?rev=809872&view=rev
Log:
CAMEL-1933: Overhaul of JMX. Introduced ManagementAware for fine grained management of camel components outside camel-core.

Added:
    camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ManagementAware.java   (with props)
Removed:
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/mbean/ManagedTimerEndpoint.java
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/component/timer/TimerEndpoint.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementNamingStrategy.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/ManagedManagementStrategy.java
    camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ManagementNamingStrategy.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/management/CustomEndpoint.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationCustomMBeanTest.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/component/timer/TimerEndpoint.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/component/timer/TimerEndpoint.java?rev=809872&r1=809871&r2=809872&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/component/timer/TimerEndpoint.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/component/timer/TimerEndpoint.java Tue Sep  1 07:51:20 2009
@@ -24,13 +24,17 @@
 import org.apache.camel.Producer;
 import org.apache.camel.RuntimeCamelException;
 import org.apache.camel.impl.DefaultEndpoint;
+import org.apache.camel.spi.ManagementAware;
+import org.springframework.jmx.export.annotation.ManagedAttribute;
+import org.springframework.jmx.export.annotation.ManagedResource;
 
 /**
  * Represents a timer endpoint that can generate periodic inbound PojoExchanges.
  *
  * @version $Revision$
  */
-public class TimerEndpoint extends DefaultEndpoint {
+@ManagedResource(description = "Managed Timer Endpoint")
+public class TimerEndpoint extends DefaultEndpoint implements ManagementAware<TimerEndpoint> {
     private String timerName;
     private Date time;
     private long period = 1000;
@@ -65,6 +69,11 @@
         return new TimerConsumer(this, processor);
     }
 
+    public Object getManagedObject(TimerEndpoint object) {
+        return this;
+    }
+
+    @ManagedAttribute(description = "Timer Name")
     public String getTimerName() {
         if (timerName == null) {
             timerName = getEndpointUri();
@@ -72,38 +81,47 @@
         return timerName;
     }
 
+    @ManagedAttribute(description = "Timer Name")
     public void setTimerName(String timerName) {
         this.timerName = timerName;
     }
 
+    @ManagedAttribute(description = "Timer Daemon")
     public boolean isDaemon() {
         return daemon;
     }
 
+    @ManagedAttribute(description = "Timer Daemon")
     public void setDaemon(boolean daemon) {
         this.daemon = daemon;
     }
 
+    @ManagedAttribute(description = "Timer Delay")
     public long getDelay() {
         return delay;
     }
 
+    @ManagedAttribute(description = "Timer Delay")
     public void setDelay(long delay) {
         this.delay = delay;
     }
 
+    @ManagedAttribute(description = "Timer FixedRate")
     public boolean isFixedRate() {
         return fixedRate;
     }
 
+    @ManagedAttribute(description = "Timer FixedRate")
     public void setFixedRate(boolean fixedRate) {
         this.fixedRate = fixedRate;
     }
 
+    @ManagedAttribute(description = "Timer Period")
     public long getPeriod() {
         return period;
     }
 
+    @ManagedAttribute(description = "Timer Period")
     public void setPeriod(long period) {
         this.period = period;
     }
@@ -116,6 +134,7 @@
         this.time = time;
     }
 
+    @ManagedAttribute(description = "Singleton")
     public boolean isSingleton() {
         return true;
     }
@@ -130,4 +149,15 @@
     public void setTimer(Timer timer) {
         this.timer = timer;
     }
+
+    @ManagedAttribute(description = "Camel id")
+    public String getCamelId() {
+        return this.getCamelContext().getName();
+    }
+
+    @ManagedAttribute(description = "Endpoint Uri")
+    public String getEndpointUri() {
+        return super.getEndpointUri();
+    }
+
 }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java?rev=809872&r1=809871&r2=809872&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementLifecycleStrategy.java Tue Sep  1 07:51:20 2009
@@ -16,13 +16,11 @@
  */
 package org.apache.camel.management;
 
-import java.lang.reflect.Method;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import javax.management.JMException;
-import javax.management.ObjectName;
 
 import org.apache.camel.CamelContext;
 import org.apache.camel.Component;
@@ -32,7 +30,6 @@
 import org.apache.camel.Producer;
 import org.apache.camel.Route;
 import org.apache.camel.Service;
-import org.apache.camel.component.timer.TimerEndpoint;
 import org.apache.camel.impl.EventDrivenConsumerRoute;
 import org.apache.camel.impl.ScheduledPollConsumer;
 import org.apache.camel.management.mbean.ManagedBrowsableEndpoint;
@@ -48,7 +45,6 @@
 import org.apache.camel.management.mbean.ManagedScheduledPollConsumer;
 import org.apache.camel.management.mbean.ManagedSendProcessor;
 import org.apache.camel.management.mbean.ManagedThrottler;
-import org.apache.camel.management.mbean.ManagedTimerEndpoint;
 import org.apache.camel.management.mbean.ManagedTracer;
 import org.apache.camel.model.AOPDefinition;
 import org.apache.camel.model.InterceptDefinition;
@@ -61,8 +57,8 @@
 import org.apache.camel.processor.Throttler;
 import org.apache.camel.processor.interceptor.Tracer;
 import org.apache.camel.spi.BrowsableEndpoint;
-import org.apache.camel.spi.ClassResolver;
 import org.apache.camel.spi.LifecycleStrategy;
+import org.apache.camel.spi.ManagementAware;
 import org.apache.camel.spi.ManagementStrategy;
 import org.apache.camel.spi.RouteContext;
 import org.apache.camel.util.KeyValueHolder;
@@ -80,7 +76,6 @@
 public class DefaultManagementLifecycleStrategy implements LifecycleStrategy, Service {
 
     private static final Log LOG = LogFactory.getLog(DefaultManagementLifecycleStrategy.class);
-    private static final String MANAGED_RESOURCE_CLASSNAME = "org.springframework.jmx.export.annotation.ManagedResource";
     private final Map<Processor, KeyValueHolder<ProcessorDefinition, InstrumentationProcessor>> wrappedProcessors =
             new HashMap<Processor, KeyValueHolder<ProcessorDefinition, InstrumentationProcessor>>();
     private final CamelContext context;
@@ -163,82 +158,41 @@
             return;
         }
 
-        // see if the spring-jmx is on the classpath
-        Class annotationClass = resolveManagedAnnotation(endpoint);
-        if (annotationClass == null) {
-            // no its not so register the endpoint as a new managed endpoint
-            registerEndpointAsManagedEndpoint(endpoint);
-            return;
-        }
-
-        // see if the endpoint have been annotation with a spring JMX annotation
-        Object annotation = endpoint.getClass().getAnnotation(annotationClass);
-        if (annotation == null) {
-            // no its not so register the endpoint as a new managed endpoint
-            registerEndpointAsManagedEndpoint(endpoint);
-        } else {
-            // there is already a spring JMX annotation so attempt to register it
-            attemptToRegisterManagedResource(endpoint, annotation);
-        }
-    }
-
-    public void onEndpointRemove(Endpoint endpoint) {
-        // the agent hasn't been started
-        if (!initialized) {
-            return;
-        }
-
         try {
-            ManagedEndpoint me;
-            if (endpoint instanceof BrowsableEndpoint) {
+            Object me;
+            if (endpoint instanceof ManagementAware) {
+                me = ((ManagementAware) endpoint).getManagedObject(endpoint);
+            } else if (endpoint instanceof BrowsableEndpoint) {
                 me = new ManagedBrowsableEndpoint((BrowsableEndpoint) endpoint);
-            } else if (endpoint instanceof TimerEndpoint) {
-                me = new ManagedTimerEndpoint((TimerEndpoint) endpoint);
             } else {
                 me = new ManagedEndpoint(endpoint);
             }
-            getStrategy().unmanageObject(me);
+            getStrategy().manageObject(me);
         } catch (Exception e) {
-            LOG.warn("Could not unregister Endpoint MBean for uri: " + endpoint.getEndpointUri(), e);
+            LOG.warn("Could not register Endpoint MBean for uri: " + endpoint.getEndpointUri(), e);
         }
     }
 
-    private Class resolveManagedAnnotation(Endpoint endpoint) {
-        CamelContext context = endpoint.getCamelContext();
-
-        ClassResolver resolver = context.getClassResolver();
-        return resolver.resolveClass(MANAGED_RESOURCE_CLASSNAME);
-    }
-
-    private void attemptToRegisterManagedResource(Endpoint endpoint, Object annotation) {
-        try {
-            Method method = annotation.getClass().getMethod("objectName");
-            String name = (String) method.invoke(annotation);
-            ObjectName objectName = ObjectName.getInstance(name);
-            getStrategy().manageNamedObject(endpoint, objectName);
-        } catch (Exception e) {
-            if (LOG.isTraceEnabled()) {
-                LOG.trace("objectName method not present on endpoint, wrapping endpoint in ManagedEndpoint instead: " + endpoint);
-            }
-            registerEndpointAsManagedEndpoint(endpoint);
+    @SuppressWarnings("unchecked")
+    public void onEndpointRemove(Endpoint endpoint) {
+        // the agent hasn't been started
+        if (!initialized) {
+            return;
         }
-    }
 
-    private void registerEndpointAsManagedEndpoint(Endpoint endpoint) {
         try {
-            ManagedEndpoint me;
-            if (endpoint instanceof BrowsableEndpoint) {
+            Object me;
+            if (endpoint instanceof ManagementAware) {
+                me = ((ManagementAware) endpoint).getManagedObject(endpoint);
+            } else if (endpoint instanceof BrowsableEndpoint) {
                 me = new ManagedBrowsableEndpoint((BrowsableEndpoint) endpoint);
-            } else if (endpoint instanceof TimerEndpoint) {
-                me = new ManagedTimerEndpoint((TimerEndpoint) endpoint);
             } else {
                 me = new ManagedEndpoint(endpoint);
             }
-            getStrategy().manageObject(me);
+            getStrategy().unmanageObject(me);
         } catch (Exception e) {
-            LOG.warn("Could not register Endpoint MBean for uri: " + endpoint.getEndpointUri(), e);
+            LOG.warn("Could not unregister Endpoint MBean for uri: " + endpoint.getEndpointUri(), e);
         }
-
     }
 
     public void onServiceAdd(CamelContext context, Service service) {

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementNamingStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementNamingStrategy.java?rev=809872&r1=809871&r2=809872&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementNamingStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/DefaultManagementNamingStrategy.java Tue Sep  1 07:51:20 2009
@@ -22,20 +22,14 @@
 import javax.management.ObjectName;
 
 import org.apache.camel.CamelContext;
+import org.apache.camel.Component;
 import org.apache.camel.Consumer;
 import org.apache.camel.Endpoint;
 import org.apache.camel.Processor;
 import org.apache.camel.Route;
-import org.apache.camel.management.mbean.ManagedComponent;
-import org.apache.camel.management.mbean.ManagedConsumer;
-import org.apache.camel.management.mbean.ManagedEndpoint;
-import org.apache.camel.management.mbean.ManagedProcessor;
-import org.apache.camel.management.mbean.ManagedRoute;
-import org.apache.camel.management.mbean.ManagedService;
-import org.apache.camel.management.mbean.ManagedTracer;
 import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.spi.InterceptStrategy;
 import org.apache.camel.spi.ManagementNamingStrategy;
-import org.apache.camel.spi.RouteContext;
 
 /**
  * Naming strategy used when registering MBeans.
@@ -71,7 +65,7 @@
         }
     }
 
-    public ObjectName getObjectName(CamelContext context) throws MalformedObjectNameException {
+    public ObjectName getObjectNameForCamelContext(CamelContext context) throws MalformedObjectNameException {
         StringBuffer buffer = new StringBuffer();
         buffer.append(domainName).append(":");
         buffer.append(KEY_CONTEXT + "=").append(getContextId(context)).append(",");
@@ -80,33 +74,28 @@
         return createObjectName(buffer);
     }
 
-    public ObjectName getObjectName(ManagedEndpoint mbean) throws MalformedObjectNameException {
-        Endpoint ep = mbean.getEndpoint();
-
+    public ObjectName getObjectNameForEndpoint(Endpoint endpoint) throws MalformedObjectNameException {
         StringBuffer buffer = new StringBuffer();
         buffer.append(domainName).append(":");
-        buffer.append(KEY_CONTEXT + "=").append(getContextId(ep.getCamelContext())).append(",");
+        buffer.append(KEY_CONTEXT + "=").append(getContextId(endpoint.getCamelContext())).append(",");
         buffer.append(KEY_TYPE + "=" + TYPE_ENDPOINT + ",");
-        buffer.append(KEY_NAME + "=").append(ObjectName.quote(getEndpointId(ep)));
+        buffer.append(KEY_NAME + "=").append(ObjectName.quote(getEndpointId(endpoint)));
         return createObjectName(buffer);
     }
 
-    public ObjectName getObjectName(ManagedComponent mbean) throws MalformedObjectNameException {
+    public ObjectName getObjectNameForComponent(Component component, String name) throws MalformedObjectNameException {
         StringBuffer buffer = new StringBuffer();
         buffer.append(domainName).append(":");
-        buffer.append(KEY_CONTEXT + "=").append(getContextId(mbean.getComponent().getCamelContext())).append(",");
+        buffer.append(KEY_CONTEXT + "=").append(getContextId(component.getCamelContext())).append(",");
         buffer.append(KEY_TYPE + "=" + TYPE_COMPONENT + ",");
-        buffer.append(KEY_NAME + "=").append(ObjectName.quote(mbean.getComponentName()));
+        buffer.append(KEY_NAME + "=").append(ObjectName.quote(name));
         return createObjectName(buffer);
     }
 
-    public ObjectName getObjectName(ManagedProcessor mbean) throws MalformedObjectNameException {
-        Processor processor = mbean.getProcessor();
-        ProcessorDefinition definition = mbean.getDefinition();
-
+    public ObjectName getObjectNameForProcessor(CamelContext context, Processor processor, ProcessorDefinition definition) throws MalformedObjectNameException {
         StringBuffer buffer = new StringBuffer();
         buffer.append(domainName).append(":");
-        buffer.append(KEY_CONTEXT + "=").append(getContextId(mbean.getContext())).append(",");
+        buffer.append(KEY_CONTEXT + "=").append(getContextId(context)).append(",");
         buffer.append(KEY_TYPE + "=").append(TYPE_PROCESSOR).append(",");
 
         if (definition.hasCustomIdAssigned()) {
@@ -122,12 +111,10 @@
         return createObjectName(buffer);
     }
 
-    public ObjectName getObjectName(ManagedConsumer mbean) throws MalformedObjectNameException {
-        Consumer consumer = mbean.getConsumer();
-
+    public ObjectName getObjectNameForConsumer(CamelContext context, Consumer consumer) throws MalformedObjectNameException {
         StringBuffer buffer = new StringBuffer();
         buffer.append(domainName).append(":");
-        buffer.append(KEY_CONTEXT + "=").append(getContextId(mbean.getContext())).append(",");
+        buffer.append(KEY_CONTEXT + "=").append(getContextId(context)).append(",");
         buffer.append(KEY_TYPE + "=").append(TYPE_CONSUMER).append(",");
         buffer.append(KEY_NAME + "=")
             .append(consumer.getClass().getSimpleName())
@@ -135,24 +122,18 @@
         return createObjectName(buffer);
     }
 
-    public ObjectName getObjectName(ManagedService mbean) throws MalformedObjectNameException {
-        // not supported
-        return null;
-    }
-
-    public ObjectName getObjectName(ManagedTracer mbean) throws MalformedObjectNameException {
+    public ObjectName getObjectNameForTracer(CamelContext context, InterceptStrategy tracer) throws MalformedObjectNameException {
         StringBuffer buffer = new StringBuffer();
         buffer.append(domainName).append(":");
-        buffer.append(KEY_CONTEXT + "=").append(getContextId(mbean.getCamelContext())).append(",");
+        buffer.append(KEY_CONTEXT + "=").append(getContextId(context)).append(",");
         buffer.append(KEY_TYPE + "=" + TYPE_TRACER + ",");
         buffer.append(KEY_NAME + "=")
             .append("Tracer")
-            .append("(").append(getIdentityHashCode(mbean.getTracer())).append(")");
+            .append("(").append(getIdentityHashCode(tracer)).append(")");
         return createObjectName(buffer);
     }
 
-    public ObjectName getObjectName(ManagedRoute mbean) throws MalformedObjectNameException {
-        Route route = mbean.getRoute();
+    public ObjectName getObjectNameForRoute(Route route) throws MalformedObjectNameException {
         Endpoint ep = route.getEndpoint();
         String id = route.getId();
 
@@ -164,21 +145,6 @@
         return createObjectName(buffer);
     }
 
-    @Deprecated
-    public ObjectName getObjectName(RouteContext routeContext, ProcessorDefinition processor)
-        throws MalformedObjectNameException {
-
-        Endpoint ep = routeContext.getEndpoint();
-        String nodeId = processor.idOrCreate(routeContext.getCamelContext().getNodeIdFactory());
-
-        StringBuffer buffer = new StringBuffer();
-        buffer.append(domainName).append(":");
-        buffer.append(KEY_CONTEXT + "=").append(getContextId(ep.getCamelContext())).append(",");
-        buffer.append(KEY_TYPE + "=" + TYPE_PROCESSOR + ",");
-        buffer.append(KEY_NAME + "=").append(ObjectName.quote(nodeId));
-        return createObjectName(buffer);
-    }
-
     public String getDomainName() {
         return domainName;
     }

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/ManagedManagementStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/ManagedManagementStrategy.java?rev=809872&r1=809871&r2=809872&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/ManagedManagementStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/ManagedManagementStrategy.java Tue Sep  1 07:51:20 2009
@@ -18,6 +18,7 @@
 
 import javax.management.ObjectName;
 
+import org.apache.camel.Endpoint;
 import org.apache.camel.management.mbean.ManagedCamelContext;
 import org.apache.camel.management.mbean.ManagedComponent;
 import org.apache.camel.management.mbean.ManagedConsumer;
@@ -71,37 +72,39 @@
 
         if (managedObject instanceof ManagedCamelContext) {
             ManagedCamelContext mcc = (ManagedCamelContext) managedObject;
-            objectName = getManagementNamingStrategy().getObjectName(mcc.getContext());
+            objectName = getManagementNamingStrategy().getObjectNameForCamelContext(mcc.getContext());
         }
 
         if (managedObject instanceof ManagedComponent) {
             ManagedComponent mc = (ManagedComponent) managedObject;
-            objectName = getManagementNamingStrategy().getObjectName(mc);
+            objectName = getManagementNamingStrategy().getObjectNameForComponent(mc.getComponent(), mc.getComponentName());
         }
 
         if (managedObject instanceof ManagedEndpoint) {
             ManagedEndpoint me = (ManagedEndpoint) managedObject;
-            objectName = getManagementNamingStrategy().getObjectName(me);
+            objectName = getManagementNamingStrategy().getObjectNameForEndpoint(me.getEndpoint());
+        } else if (managedObject instanceof Endpoint) {
+            objectName = getManagementNamingStrategy().getObjectNameForEndpoint((Endpoint) managedObject);
         }
 
         if (managedObject instanceof ManagedRoute) {
             ManagedRoute mr = (ManagedRoute) managedObject;
-            objectName = getManagementNamingStrategy().getObjectName(mr);
+            objectName = getManagementNamingStrategy().getObjectNameForRoute(mr.getRoute());
         }
 
         if (managedObject instanceof ManagedProcessor) {
             ManagedProcessor mp = (ManagedProcessor) managedObject;
-            objectName = getManagementNamingStrategy().getObjectName(mp);
+            objectName = getManagementNamingStrategy().getObjectNameForProcessor(mp.getContext(), mp.getProcessor(), mp.getDefinition());
         }
 
         if (managedObject instanceof ManagedConsumer) {
             ManagedConsumer ms = (ManagedConsumer) managedObject;
-            objectName = getManagementNamingStrategy().getObjectName(ms);
+            objectName = getManagementNamingStrategy().getObjectNameForConsumer(ms.getContext(), ms.getConsumer());
         }
 
         if (managedObject instanceof ManagedTracer) {
             ManagedTracer mt = (ManagedTracer) managedObject;
-            objectName = getManagementNamingStrategy().getObjectName(mt);
+            objectName = getManagementNamingStrategy().getObjectNameForTracer(mt.getCamelContext(), mt.getTracer());
         }
 
         return nameType.cast(objectName);

Added: camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ManagementAware.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ManagementAware.java?rev=809872&view=auto
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ManagementAware.java (added)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ManagementAware.java Tue Sep  1 07:51:20 2009
@@ -0,0 +1,40 @@
+/**
+ * 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.spi;
+
+/**
+ * An interface to represent an object being managed.
+ * <p/>
+ * This allows you to gain fine grained control of manageing objects with Camel.
+ * For example various Camel components will implement this interface to provide
+ * management to their endpoints and consumers.
+ * <p/>
+ * Camel will by default use generic management objects if objects do not implement
+ * this interface. These defaults are located in <tt>org.apache.camel.management.mbean</tt>.
+ *
+ * @version $Revision$
+ */
+public interface ManagementAware<T> {
+
+    /**
+     * Gets the managed object
+     *
+     * @param object the object to be managed
+     * @return the managed object
+     */
+    Object getManagedObject(T object);
+}

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

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

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ManagementNamingStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ManagementNamingStrategy.java?rev=809872&r1=809871&r2=809872&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ManagementNamingStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/spi/ManagementNamingStrategy.java Tue Sep  1 07:51:20 2009
@@ -20,13 +20,11 @@
 import javax.management.ObjectName;
 
 import org.apache.camel.CamelContext;
-import org.apache.camel.management.mbean.ManagedComponent;
-import org.apache.camel.management.mbean.ManagedConsumer;
-import org.apache.camel.management.mbean.ManagedEndpoint;
-import org.apache.camel.management.mbean.ManagedProcessor;
-import org.apache.camel.management.mbean.ManagedRoute;
-import org.apache.camel.management.mbean.ManagedService;
-import org.apache.camel.management.mbean.ManagedTracer;
+import org.apache.camel.Component;
+import org.apache.camel.Consumer;
+import org.apache.camel.Endpoint;
+import org.apache.camel.Processor;
+import org.apache.camel.Route;
 import org.apache.camel.model.ProcessorDefinition;
 
 /**
@@ -34,28 +32,18 @@
  */
 public interface ManagementNamingStrategy {
 
-    ObjectName getObjectName(CamelContext context) throws MalformedObjectNameException;
+    ObjectName getObjectNameForCamelContext(CamelContext context) throws MalformedObjectNameException;
 
-    ObjectName getObjectName(ManagedComponent mbean) throws MalformedObjectNameException;
+    ObjectName getObjectNameForComponent(Component component, String name) throws MalformedObjectNameException;
 
-    ObjectName getObjectName(ManagedEndpoint mbean) throws MalformedObjectNameException;
+    ObjectName getObjectNameForEndpoint(Endpoint endpoint) throws MalformedObjectNameException;
 
-    ObjectName getObjectName(ManagedProcessor mbean) throws MalformedObjectNameException;
+    ObjectName getObjectNameForProcessor(CamelContext context, Processor processor, ProcessorDefinition definition) throws MalformedObjectNameException;
 
-    ObjectName getObjectName(ManagedRoute mbean) throws MalformedObjectNameException;
+    ObjectName getObjectNameForRoute(Route route) throws MalformedObjectNameException;
 
-    ObjectName getObjectName(ManagedConsumer mbean) throws MalformedObjectNameException;
+    ObjectName getObjectNameForConsumer(CamelContext context, Consumer consumer) throws MalformedObjectNameException;
 
-    ObjectName getObjectName(ManagedTracer mbean) throws MalformedObjectNameException;
-
-    /**
-     * @deprecated
-     */
-    ObjectName getObjectName(ManagedService mbean) throws MalformedObjectNameException;
-
-    /**
-     * @deprecated
-     */
-    ObjectName getObjectName(RouteContext routeContext, ProcessorDefinition processor) throws MalformedObjectNameException;
+    ObjectName getObjectNameForTracer(CamelContext context, InterceptStrategy tracer) throws MalformedObjectNameException;
 
 }

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/management/CustomEndpoint.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/management/CustomEndpoint.java?rev=809872&r1=809871&r2=809872&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/management/CustomEndpoint.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/management/CustomEndpoint.java Tue Sep  1 07:51:20 2009
@@ -18,6 +18,7 @@
 
 import org.apache.camel.Component;
 import org.apache.camel.component.mock.MockEndpoint;
+import org.apache.camel.spi.ManagementAware;
 import org.springframework.jmx.export.annotation.ManagedAttribute;
 import org.springframework.jmx.export.annotation.ManagedResource;
 
@@ -26,17 +27,21 @@
  * and must be declared a public class otherwise the mbean server connection cannot access its methods.
  */
 // START SNIPPET: e1
-@ManagedResource(objectName = "testdomain:name=customEndpoint")
-public class CustomEndpoint extends MockEndpoint {
+@ManagedResource(description = "Our custom managed endpoint")
+public class CustomEndpoint extends MockEndpoint implements ManagementAware<CustomEndpoint> {
 
     public CustomEndpoint(final String endpointUri, final Component component) {
         super(endpointUri, component);
     }
 
+    public Object getManagedObject(CustomEndpoint object) {
+        return this;
+    }
+
     public boolean isSingleton() {
         return true;
     }
-    
+
     protected String createEndpointUri() {
         return "custom";
     }
@@ -45,5 +50,10 @@
     public String getFoo() {
         return "bar";
     }
+
+    @ManagedAttribute
+    public String getEndpointUri() {
+        return super.getEndpointUri();
+    }
 }
 // END SNIPPET: e1

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationCustomMBeanTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationCustomMBeanTest.java?rev=809872&r1=809871&r2=809872&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationCustomMBeanTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationCustomMBeanTest.java Tue Sep  1 07:51:20 2009
@@ -16,6 +16,7 @@
  */
 package org.apache.camel.management;
 
+import java.util.Iterator;
 import java.util.Map;
 import java.util.Set;
 import javax.management.ObjectName;
@@ -24,7 +25,6 @@
 import org.apache.camel.Endpoint;
 import org.apache.camel.builder.RouteBuilder;
 import org.apache.camel.component.direct.DirectEndpoint;
-import org.apache.camel.impl.DefaultCamelContext;
 import org.apache.camel.impl.DefaultComponent;
 
 /**
@@ -33,6 +33,17 @@
  */
 public class JmxInstrumentationCustomMBeanTest extends JmxInstrumentationUsingDefaultsTest {
 
+    protected CamelContext createCamelContext() throws Exception {
+        CamelContext context = super.createCamelContext();
+
+        context.addComponent("custom", new CustomComponent());
+
+        DefaultManagementNamingStrategy naming = (DefaultManagementNamingStrategy) context.getManagementStrategy().getManagementNamingStrategy();
+        naming.setHostName("localhost");
+        naming.setDomainName("org.apache.camel");
+        return context;
+    }
+
     public void testCustomEndpoint() throws Exception {
         if (!canRunOnThisPlatform()) {
             return;
@@ -43,12 +54,24 @@
             assertEquals(domainName, mbsc.getDefaultDomain());
         }
 
-        resolveMandatoryEndpoint("custom:end", CustomEndpoint.class);
-        ObjectName objName = new ObjectName("testdomain:name=customEndpoint");
+        resolveMandatoryEndpoint("custom://end", CustomEndpoint.class);
+
+        Set s = mbsc.queryNames(new ObjectName(domainName + ":type=endpoints,*"), null);
+        assertEquals("Could not find 2 endpoints: " + s, 2, s.size());
+
+        // get custom
+        Iterator<ObjectName> it = s.iterator();
+        ObjectName on1 = it.next();
+        ObjectName on2 = it.next();
 
-        assertEquals("bar", mbsc.getAttribute(objName, "Foo"));
+        if (on1.getCanonicalName().contains("custom")) {
+            assertEquals("bar", mbsc.getAttribute(on1, "Foo"));
+        } else {
+            assertEquals("bar", mbsc.getAttribute(on2, "Foo"));
+        }
     }
 
+    @SuppressWarnings("unchecked")
     public void testManagedEndpoint() throws Exception {
         if (!canRunOnThisPlatform()) {
             return;
@@ -63,10 +86,7 @@
 
         ObjectName objName = new ObjectName(domainName + ":type=endpoints,*");
         Set<ObjectName> s = mbsc.queryNames(objName, null);
-
-        ObjectName dynamicallyGeneratedObjName = s.iterator().next();
-
-        assertEquals("direct://start", mbsc.getAttribute(dynamicallyGeneratedObjName, "EndpointUri"));
+        assertEquals(2, s.size());
     }
 
     public void testCounters() throws Exception {
@@ -83,6 +103,30 @@
         verifyCounter(mbsc, new ObjectName(domainName + ":type=routes,*"));
     }
 
+    public void testMBeansRegistered() throws Exception {
+        if (!canRunOnThisPlatform()) {
+            return;
+        }
+
+        if (System.getProperty(JmxSystemPropertyKeys.USE_PLATFORM_MBS) != null
+                && !Boolean.getBoolean(JmxSystemPropertyKeys.USE_PLATFORM_MBS)) {
+            assertEquals(domainName, mbsc.getDefaultDomain());
+        }
+
+        Set s = mbsc.queryNames(new ObjectName(domainName + ":type=endpoints,*"), null);
+        assertEquals("Could not find 2 endpoints: " + s, 2, s.size());
+
+        s = mbsc.queryNames(new ObjectName(domainName + ":type=context,*"), null);
+        assertEquals("Could not find 1 context: " + s, 1, s.size());
+
+        s = mbsc.queryNames(new ObjectName(domainName + ":type=processors,*"), null);
+        assertEquals("Could not find 2 processors: " + s, 2, s.size());
+
+        s = mbsc.queryNames(new ObjectName(domainName + ":type=routes,*"), null);
+        assertEquals("Could not find 1 route: " + s, 1, s.size());
+    }
+
+
     @Override
     protected RouteBuilder createRouteBuilder() {
         return new RouteBuilder() {
@@ -94,14 +138,6 @@
         };
     }
 
-    @Override
-    protected CamelContext createCamelContext() throws Exception {
-        CamelContext context = new DefaultCamelContext(createRegistry());
-        context.addComponent("custom", new CustomComponent());
-
-        return context;
-    }
-
     private class CustomComponent extends DefaultComponent {
         protected Endpoint createEndpoint(final String uri, final String remaining, final Map parameters) throws Exception {
             return new CustomEndpoint("custom", this);