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/04/01 18:33:25 UTC

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

Author: davsclaus
Date: Wed Apr  1 16:33:24 2009
New Revision: 760956

URL: http://svn.apache.org/viewvc?rev=760956&view=rev
Log:
CAMEL-1450: JMX, allow custom endpoints to be managed as well if they are annotated with @ManagedResource.

Added:
    camel/trunk/camel-core/src/test/java/org/apache/camel/management/CustomEndpoint.java   (with props)
    camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationCustomMBeanTest.java   (with props)
Modified:
    camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java
    camel/trunk/camel-core/src/test/java/org/apache/camel/management/MultiInstanceProcessorTest.java

Modified: camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java?rev=760956&r1=760955&r2=760956&view=diff
==============================================================================
--- camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java (original)
+++ camel/trunk/camel-core/src/main/java/org/apache/camel/management/InstrumentationLifecycleStrategy.java Wed Apr  1 16:33:24 2009
@@ -16,12 +16,12 @@
  */
 package org.apache.camel.management;
 
+import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-
 import javax.management.JMException;
 import javax.management.MalformedObjectNameException;
 import javax.management.ObjectName;
@@ -36,6 +36,7 @@
 import org.apache.camel.model.OnExceptionDefinition;
 import org.apache.camel.model.ProcessorDefinition;
 import org.apache.camel.model.RouteDefinition;
+import org.apache.camel.spi.ClassResolver;
 import org.apache.camel.spi.InstrumentationAgent;
 import org.apache.camel.spi.LifecycleStrategy;
 import org.apache.camel.spi.RouteContext;
@@ -51,6 +52,7 @@
 public class InstrumentationLifecycleStrategy implements LifecycleStrategy {
     private static final transient Log LOG = LogFactory.getLog(InstrumentationProcessor.class);
 
+    private static final String MANAGED_RESOURCE_CLASSNAME = "org.springframework.jmx.export.annotation.ManagedResource";
     private InstrumentationAgent agent;
     private CamelNamingStrategy namingStrategy;
     private boolean initialized;
@@ -96,17 +98,62 @@
         }
     }
 
+    /**
+     * If the endpoint is an instance of ManagedResource then register it with the
+     * mbean server, if it is not then wrap the endpoint in a {@link ManagedEndpoint} and
+     * register that with the mbean server.
+     * @param endpoint the Endpoint attempted to be added
+     */
+    @SuppressWarnings("unchecked")
     public void onEndpointAdd(Endpoint endpoint) {
         // the agent hasn't been started
         if (!initialized) {
             return;
         }
 
+        Class annotationClass = resolveManagedAnnotation(endpoint);
+        if (annotationClass == null) {
+            registerEndpointAsManagedEndpoint(endpoint);
+            return;
+        }
+
+        Object annotation = endpoint.getClass().getAnnotation(annotationClass);
+        if (annotation == null) {
+            registerEndpointAsManagedEndpoint(endpoint);
+            return;
+        }
+
+        attemptToRegisterManagedResource(endpoint, annotation);
+    }
+
+    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 m = annotation.getClass().getMethod("objectName");
+
+            String objectNameStr = (String) m.invoke(annotation);
+
+            ObjectName objectName = new ObjectName(objectNameStr);
+            agent.register(endpoint, objectName);
+        } catch (Exception e) {
+            LOG.debug("objectName method not present, wrapping endpoint in ManagedEndpoint instead");
+            registerEndpointAsManagedEndpoint(endpoint);
+        }
+    }
+
+    private void registerEndpointAsManagedEndpoint(Endpoint endpoint) {
         try {
             ManagedEndpoint me = new ManagedEndpoint(endpoint);
             agent.register(me, getNamingStrategy().getObjectName(me));
         } catch (JMException e) {
-            LOG.warn("Could not register Endpoint MBean", e);
+            LOG.warn("Could not register Endpoint MBean for uri: " + endpoint.getEndpointUri(), e);
         }
     }
 

Added: 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=760956&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/management/CustomEndpoint.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/management/CustomEndpoint.java Wed Apr  1 16:33:24 2009
@@ -0,0 +1,49 @@
+/**
+ * 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.Component;
+import org.apache.camel.component.mock.MockEndpoint;
+import org.springframework.jmx.export.annotation.ManagedAttribute;
+import org.springframework.jmx.export.annotation.ManagedResource;
+
+/**
+ * CustomEndpoint is used to test {@link org.apache.camel.management.JmxInstrumentationCustomMBeanTest}
+ * 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 {
+
+    public CustomEndpoint(final String endpointUri, final Component component) {
+        super(endpointUri, component);
+    }
+
+    public boolean isSingleton() {
+        return true;
+    }
+    
+    protected String createEndpointUri() {
+        return "custom";
+    }
+
+    @ManagedAttribute
+    public String getFoo() {
+        return "bar";
+    }
+}
+// END SNIPPET: e1

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

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

Added: 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=760956&view=auto
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationCustomMBeanTest.java (added)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/management/JmxInstrumentationCustomMBeanTest.java Wed Apr  1 16:33:24 2009
@@ -0,0 +1,99 @@
+/**
+ * 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.Map;
+import java.util.Set;
+import javax.management.ObjectName;
+
+import org.apache.camel.CamelContext;
+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;
+
+/**
+ * JmxInstrumentationCustomMBeanTest will verify that all endpoints are registered
+ * with the mbean server.
+ */
+public class JmxInstrumentationCustomMBeanTest extends JmxInstrumentationUsingDefaultsTest {
+
+    public void testCustomEndpoint() throws Exception {
+        if (System.getProperty(JmxSystemPropertyKeys.USE_PLATFORM_MBS) != null
+                && !Boolean.getBoolean(JmxSystemPropertyKeys.USE_PLATFORM_MBS)) {
+            assertEquals(domainName, mbsc.getDefaultDomain());
+        }
+
+        resolveMandatoryEndpoint("custom:end", CustomEndpoint.class);
+        ObjectName objName = new ObjectName("testdomain:name=customEndpoint");
+
+        assertEquals("bar", mbsc.getAttribute(objName, "Foo"));
+    }
+
+    public void testManagedEndpoint() throws Exception {
+        if (System.getProperty(JmxSystemPropertyKeys.USE_PLATFORM_MBS) != null
+                && !Boolean.getBoolean(JmxSystemPropertyKeys.USE_PLATFORM_MBS)) {
+            assertEquals(domainName, mbsc.getDefaultDomain());
+        }
+
+        resolveMandatoryEndpoint("direct:start", DirectEndpoint.class);
+
+        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, "Uri"));
+    }
+
+    public void testCounters() throws Exception {
+        CustomEndpoint resultEndpoint = resolveMandatoryEndpoint("custom:end", CustomEndpoint.class);
+        resultEndpoint.expectedBodiesReceived("<hello>world!</hello>");
+        sendBody("direct:start", "<hello>world!</hello>");
+
+        resultEndpoint.assertIsSatisfied();
+
+        verifyCounter(mbsc, new ObjectName(domainName + ":type=routes,*"));
+        verifyCounter(mbsc, new ObjectName(domainName + ":type=processors,*"));
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() {
+        return new RouteBuilder() {
+            public void configure() {
+                // need a little delay for fast computers being able to process
+                // the exchange in 0 millis and we need to simulate a little computation time
+                from("direct:start").delay(10).to("custom:end");
+            }
+        };
+    }
+
+    @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);
+        }
+    }
+}

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

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

Modified: camel/trunk/camel-core/src/test/java/org/apache/camel/management/MultiInstanceProcessorTest.java
URL: http://svn.apache.org/viewvc/camel/trunk/camel-core/src/test/java/org/apache/camel/management/MultiInstanceProcessorTest.java?rev=760956&r1=760955&r2=760956&view=diff
==============================================================================
--- camel/trunk/camel-core/src/test/java/org/apache/camel/management/MultiInstanceProcessorTest.java (original)
+++ camel/trunk/camel-core/src/test/java/org/apache/camel/management/MultiInstanceProcessorTest.java Wed Apr  1 16:33:24 2009
@@ -51,38 +51,29 @@
 
         resolveMandatoryEndpoint("mock:end", MockEndpoint.class);
 
-        Set s = mbsc.queryNames(
-                new ObjectName(domainName + ":type=endpoints,*"), null);
+        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 + ":name=context,*"), null);
+        s = mbsc.queryNames(new ObjectName(domainName + ":name=context,*"), null);
         assertEquals("Could not find 1 context: " + s, 1, s.size());
 
-        s = mbsc.queryNames(
-                new ObjectName(domainName + ":type=processors,*"), null);
+        s = mbsc.queryNames(new ObjectName(domainName + ":type=processors,*"), null);
         assertEquals("Could not find 2 processor: " + s, 2, s.size());
 
-        s = mbsc.queryNames(
-                new ObjectName(domainName + ":type=routes,*"), null);
+        s = mbsc.queryNames(new ObjectName(domainName + ":type=routes,*"), null);
         assertEquals("Could not find 1 route: " + s, 1, s.size());
 
     }
 
     @Override
     public void testCounters() throws Exception {
-
         MockEndpoint resultEndpoint = resolveMandatoryEndpoint("mock:end", MockEndpoint.class);
-        resultEndpoint.expectedBodiesReceived(
-                new Object[] {"<hello>world!</hello>", "<hello>world!</hello>"});
+        resultEndpoint.expectedBodiesReceived("<hello>world!</hello>", "<hello>world!</hello>");
         sendBody("direct:start", "<hello>world!</hello>");
 
         resultEndpoint.assertIsSatisfied();
 
         verifyCounter(mbsc, new ObjectName(domainName + ":type=routes,*"));
-        verifyCounter(mbsc, new ObjectName(domainName + ":type=processors,nodeid=to3,*"));
-        verifyCounter(mbsc, new ObjectName(domainName + ":type=processors,nodeid=to4,*"));
-
     }
 
 }