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 2015/08/11 15:23:05 UTC

[1/6] camel git commit: CAMEL-8147: Camel JMX when using custom @ManagedResource should mixin the standard mbean operations/attributes so you get both worlds.

Repository: camel
Updated Branches:
  refs/heads/master 477c9d794 -> dd8ea2f68


CAMEL-8147: Camel JMX when using custom @ManagedResource should mixin the standard mbean operations/attributes so you get both worlds.


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/aa1fba8f
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/aa1fba8f
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/aa1fba8f

Branch: refs/heads/master
Commit: aa1fba8f4dc2ecea9569792ac3d7b56a23b969c1
Parents: 477c9d7
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Aug 11 12:45:21 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Tue Aug 11 15:30:39 2015 +0200

----------------------------------------------------------------------
 .../management/CustomRequiredModelMBean.java    |  76 ++++++++++++
 .../DefaultManagementMBeanAssembler.java        |  35 ++++--
 .../management/MaskRequiredModelMBean.java      |   1 +
 .../management/MixinRequiredModelMBean.java     | 117 +++++++++++++++++++
 .../management/ManagedCustomProcessorTest.java  |  16 ++-
 .../camel/management/ManagedResourceTest.java   |  15 +--
 6 files changed, 242 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/aa1fba8f/camel-core/src/main/java/org/apache/camel/management/CustomRequiredModelMBean.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/management/CustomRequiredModelMBean.java b/camel-core/src/main/java/org/apache/camel/management/CustomRequiredModelMBean.java
new file mode 100644
index 0000000..c8efe97
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/management/CustomRequiredModelMBean.java
@@ -0,0 +1,76 @@
+/**
+ * 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 javax.management.DynamicMBean;
+import javax.management.MBeanException;
+import javax.management.MBeanOperationInfo;
+import javax.management.ReflectionException;
+import javax.management.RuntimeOperationsException;
+import javax.management.modelmbean.ModelMBeanInfo;
+import javax.management.modelmbean.RequiredModelMBean;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link javax.management.modelmbean.RequiredModelMBean} which allows us to intercept invoking operations on the MBean.
+ * <p/>
+ * This allows us to intercept calls to custom mbeans where allows us to mix-in the standard set of mbean attributes
+ * and operations that Camel provides out of the box.
+ */
+@Deprecated
+public class CustomRequiredModelMBean extends RequiredModelMBean {
+
+    private static final Logger LOG = LoggerFactory.getLogger(CustomRequiredModelMBean.class);
+    private ModelMBeanInfo defaultMbi;
+    private DynamicMBean defaultObject;
+
+    public CustomRequiredModelMBean() throws MBeanException, RuntimeOperationsException {
+        // must have default no-arg constructor
+    }
+
+    public CustomRequiredModelMBean(ModelMBeanInfo mbi, ModelMBeanInfo defaultMbi, DynamicMBean defaultObject) throws MBeanException, RuntimeOperationsException {
+        super(mbi);
+        this.defaultMbi = defaultMbi;
+        this.defaultObject = defaultObject;
+    }
+
+    @Override
+    public Object invoke(String opName, Object[] opArgs, String[] sig) throws MBeanException, ReflectionException {
+        Object answer;
+        if (isDefaultOperation(opName)) {
+            answer = defaultObject.invoke(opName, opArgs, sig);
+        } else {
+            answer = super.invoke(opName, opArgs, sig);
+        }
+        return answer;
+    }
+
+    protected boolean isDefaultOperation(String opName) {
+        if (defaultMbi == null || defaultObject == null) {
+            return false;
+        }
+        for (MBeanOperationInfo info : defaultMbi.getOperations()) {
+            if (info.getName().equals(opName)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/aa1fba8f/camel-core/src/main/java/org/apache/camel/management/DefaultManagementMBeanAssembler.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/management/DefaultManagementMBeanAssembler.java b/camel-core/src/main/java/org/apache/camel/management/DefaultManagementMBeanAssembler.java
index abc8d44..a28bcbb 100644
--- a/camel-core/src/main/java/org/apache/camel/management/DefaultManagementMBeanAssembler.java
+++ b/camel-core/src/main/java/org/apache/camel/management/DefaultManagementMBeanAssembler.java
@@ -52,17 +52,18 @@ public class DefaultManagementMBeanAssembler implements ManagementMBeanAssembler
 
     public ModelMBean assemble(MBeanServer mBeanServer, Object obj, ObjectName name) throws JMException {
         ModelMBeanInfo mbi = null;
+        ModelMBeanInfo standardMbi = null;
+        Object custom = null;
 
         // prefer to use the managed instance if it has been annotated with JMX annotations
         if (obj instanceof ManagedInstance) {
             // there may be a custom embedded instance which have additional methods
-            Object custom = ((ManagedInstance) obj).getInstance();
+            custom = ((ManagedInstance) obj).getInstance();
             if (custom != null && ObjectHelper.hasAnnotation(custom.getClass().getAnnotations(), ManagedResource.class)) {
                 LOG.trace("Assembling MBeanInfo for: {} from custom @ManagedResource object: {}", name, custom);
-                // get the mbean info from the custom managed object
-                mbi = assembler.getMBeanInfo(null, custom, name.toString());
-                // and let the custom object be registered in JMX
-                obj = custom;
+                // get the mbean info into different groups (mbi = both, standard = standard out of the box mbi)
+                mbi = assembler.getMBeanInfo(obj, custom, name.toString());
+                standardMbi = assembler.getMBeanInfo(obj, null, name.toString());
             }
         }
 
@@ -77,14 +78,28 @@ public class DefaultManagementMBeanAssembler implements ManagementMBeanAssembler
         }
 
         RequiredModelMBean mbean;
+        RequiredModelMBean mixinMBean = null;
+
         boolean sanitize = camelContext.getManagementStrategy().getManagementAgent().getMask() != null && camelContext.getManagementStrategy().getManagementAgent().getMask();
-        if (sanitize) {
-            mbean = new MaskRequiredModelMBean(mbi, sanitize);
-        } else {
-            mbean = (RequiredModelMBean) mBeanServer.instantiate(RequiredModelMBean.class.getName());
-            mbean.setModelMBeanInfo(mbi);
+
+        // if we have a custom mbean then create a mixin mbean for the standard mbean which we would
+        // otherwise have created that contains the out of the box attributes and operations
+        // as we want a combined mbean that has both the custom and the standard
+        if (standardMbi != null) {
+            mixinMBean = (RequiredModelMBean) mBeanServer.instantiate(RequiredModelMBean.class.getName());
+            mixinMBean.setModelMBeanInfo(standardMbi);
+            try {
+                mixinMBean.setManagedResource(obj, "ObjectReference");
+            } catch (InvalidTargetObjectTypeException e) {
+                throw new JMException(e.getMessage());
+            }
+            // use custom as the object to call
+            obj = custom;
         }
 
+        // use a mixin mbean model to combine the custom and standard (custom is optional)
+        mbean = new MixinRequiredModelMBean(mbi, sanitize, standardMbi, mixinMBean);
+
         try {
             mbean.setManagedResource(obj, "ObjectReference");
         } catch (InvalidTargetObjectTypeException e) {

http://git-wip-us.apache.org/repos/asf/camel/blob/aa1fba8f/camel-core/src/main/java/org/apache/camel/management/MaskRequiredModelMBean.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/management/MaskRequiredModelMBean.java b/camel-core/src/main/java/org/apache/camel/management/MaskRequiredModelMBean.java
index 697ed3e..50966e3 100644
--- a/camel-core/src/main/java/org/apache/camel/management/MaskRequiredModelMBean.java
+++ b/camel-core/src/main/java/org/apache/camel/management/MaskRequiredModelMBean.java
@@ -35,6 +35,7 @@ import org.slf4j.LoggerFactory;
  * For example if mask has been enabled on JMX, then we use this implementation
  * to hide sensitive information from the returned JMX attributes / operations.
  */
+@Deprecated
 public class MaskRequiredModelMBean extends RequiredModelMBean {
 
     private static final Logger LOG = LoggerFactory.getLogger(MaskRequiredModelMBean.class);

http://git-wip-us.apache.org/repos/asf/camel/blob/aa1fba8f/camel-core/src/main/java/org/apache/camel/management/MixinRequiredModelMBean.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/management/MixinRequiredModelMBean.java b/camel-core/src/main/java/org/apache/camel/management/MixinRequiredModelMBean.java
new file mode 100644
index 0000000..b949d0e
--- /dev/null
+++ b/camel-core/src/main/java/org/apache/camel/management/MixinRequiredModelMBean.java
@@ -0,0 +1,117 @@
+/**
+ * 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 javax.management.Descriptor;
+import javax.management.DynamicMBean;
+import javax.management.MBeanException;
+import javax.management.MBeanOperationInfo;
+import javax.management.ReflectionException;
+import javax.management.RuntimeOperationsException;
+import javax.management.modelmbean.ModelMBeanInfo;
+import javax.management.modelmbean.RequiredModelMBean;
+
+import org.apache.camel.util.ObjectHelper;
+import org.apache.camel.util.URISupport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A {@link javax.management.modelmbean.RequiredModelMBean} which allows us to intercept invoking operations on the MBean.
+ * <p/>
+ * This allows us to intercept calls to custom mbeans where allows us to mix-in the standard set of mbean attributes
+ * and operations that Camel provides out of the box.
+ * <p/>
+ * For example if mask has been enabled on JMX, then we use this implementation
+ * to hide sensitive information from the returned JMX attributes / operations.
+ */
+public class MixinRequiredModelMBean extends RequiredModelMBean {
+
+    private static final Logger LOG = LoggerFactory.getLogger(MixinRequiredModelMBean.class);
+    private boolean mask;
+    private ModelMBeanInfo defaultMbi;
+    private DynamicMBean defaultObject;
+
+    public MixinRequiredModelMBean() throws MBeanException, RuntimeOperationsException {
+        // must have default no-arg constructor
+    }
+
+    public MixinRequiredModelMBean(ModelMBeanInfo mbi, boolean mask, ModelMBeanInfo defaultMbi, DynamicMBean defaultObject) throws MBeanException, RuntimeOperationsException {
+        super(mbi);
+        this.mask = mask;
+        this.defaultMbi = defaultMbi;
+        this.defaultObject = defaultObject;
+    }
+
+    public boolean isMask() {
+        return mask;
+    }
+
+    @Override
+    public Object invoke(String opName, Object[] opArgs, String[] sig) throws MBeanException, ReflectionException {
+        Object answer;
+        if (defaultMbi != null && defaultObject != null && isDefaultOperation(opName)) {
+            answer = defaultObject.invoke(opName, opArgs, sig);
+        } else {
+            answer = super.invoke(opName, opArgs, sig);
+        }
+        // mask the answer if enabled and it was a String type (we cannot mask other types)
+        if (mask && answer instanceof String && ObjectHelper.isNotEmpty(answer) && isMaskOperation(opName)) {
+            answer = mask(opName, (String) answer);
+        }
+        return answer;
+    }
+
+    protected boolean isDefaultOperation(String opName) {
+        for (MBeanOperationInfo info : defaultMbi.getOperations()) {
+            if (info.getName().equals(opName)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    protected boolean isMaskOperation(String opName) {
+        for (MBeanOperationInfo info : getMBeanInfo().getOperations()) {
+            if (info.getName().equals(opName)) {
+                Descriptor desc = info.getDescriptor();
+                if (desc != null) {
+                    Object val = desc.getFieldValue("mask");
+                    return val != null && "true".equals(val);
+                }
+            }
+        }
+        return false;
+    }
+
+    /**
+     * Masks the returned value from invoking the operation
+     *
+     * @param opName  the operation name invoked
+     * @param value   the current value
+     * @return the masked value
+     */
+    protected String mask(String opName, String value) {
+        // use sanitize uri which will mask sensitive information
+        String answer = URISupport.sanitizeUri(value);
+        if (LOG.isTraceEnabled()) {
+            LOG.trace("Masking JMX operation: {}.{} value: {} -> {}",
+                    new Object[]{getMBeanInfo().getClassName(), opName, value, answer});
+        }
+        return answer;
+    }
+}

http://git-wip-us.apache.org/repos/asf/camel/blob/aa1fba8f/camel-core/src/test/java/org/apache/camel/management/ManagedCustomProcessorTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/management/ManagedCustomProcessorTest.java b/camel-core/src/test/java/org/apache/camel/management/ManagedCustomProcessorTest.java
index 4d6138e..18e5209 100644
--- a/camel-core/src/test/java/org/apache/camel/management/ManagedCustomProcessorTest.java
+++ b/camel-core/src/test/java/org/apache/camel/management/ManagedCustomProcessorTest.java
@@ -25,6 +25,7 @@ import org.apache.camel.Processor;
 import org.apache.camel.api.management.ManagedAttribute;
 import org.apache.camel.api.management.ManagedResource;
 import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.support.ServiceSupport;
 
 /**
  * @version 
@@ -57,6 +58,9 @@ public class ManagedCustomProcessorTest extends ManagementTestSupport {
         getMockEndpoint("mock:result").expectedHeaderReceived("foo", "changed");
         template.sendBody("direct:start", "Bye World");
         assertMockEndpointsSatisfied();
+
+        String state = (String) mbeanServer.getAttribute(on, "State");
+        assertEquals("Started", state);
     }
 
     @Override
@@ -73,7 +77,7 @@ public class ManagedCustomProcessorTest extends ManagementTestSupport {
 
     // START SNIPPET: e1
     @ManagedResource(description = "My Managed Component")
-    public static class MyCustomProcessor implements Processor {
+    public static class MyCustomProcessor extends ServiceSupport implements Processor {
         private String foo = "hey";
 
         @ManagedAttribute
@@ -89,6 +93,16 @@ public class ManagedCustomProcessorTest extends ManagementTestSupport {
         public void process(Exchange exchange) throws Exception {
             exchange.getIn().setHeader("foo", getFoo());
         }
+
+        @Override
+        protected void doStart() throws Exception {
+            // noop
+        }
+
+        @Override
+        protected void doStop() throws Exception {
+            // noop
+        }
     }
     // END SNIPPET: e1
 

http://git-wip-us.apache.org/repos/asf/camel/blob/aa1fba8f/camel-core/src/test/java/org/apache/camel/management/ManagedResourceTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/management/ManagedResourceTest.java b/camel-core/src/test/java/org/apache/camel/management/ManagedResourceTest.java
index ce4de16..41e40fe 100644
--- a/camel-core/src/test/java/org/apache/camel/management/ManagedResourceTest.java
+++ b/camel-core/src/test/java/org/apache/camel/management/ManagedResourceTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.camel.management;
 
-import javax.management.AttributeNotFoundException;
 import javax.management.MBeanServer;
 import javax.management.ObjectName;
 
@@ -87,11 +86,13 @@ public class ManagedResourceTest extends ManagementTestSupport {
         camelsSeenCount = (Integer) mBeanServer.getAttribute(onManagedBean, "CamelsSeenCount");
         TestCase.assertEquals(0, camelsSeenCount);
 
-        try {
-            mBeanServer.getAttribute(onManagedBean, "CamelId");
-            fail("The CamelId attribute should not exist");
-        } catch (AttributeNotFoundException e) {
-            // expected
-        }
+        String camelId = (String) mBeanServer.getAttribute(onManagedBean, "CamelId");
+        assertEquals(context.getName(), camelId);
+
+        String state = (String) mBeanServer.getAttribute(onManagedBean, "State");
+        assertEquals("Started", state);
+
+        String fqn = (String) mBeanServer.getAttribute(onManagedBean, "BeanClassName");
+        assertEquals(MyManagedBean.class.getCanonicalName(), fqn);
     }
 }


[4/6] camel git commit: CAMEL-8147: Camel JMX when using custom @ManagedResource should mixin the standard mbean operations/attributes so you get both worlds.

Posted by da...@apache.org.
CAMEL-8147: Camel JMX when using custom @ManagedResource should mixin the standard mbean operations/attributes so you get both worlds.


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/f28399ff
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/f28399ff
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/f28399ff

Branch: refs/heads/master
Commit: f28399ff35dec089db091dc9c9e03cecbb356b89
Parents: aa1fba8
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Aug 11 14:15:08 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Tue Aug 11 15:30:40 2015 +0200

----------------------------------------------------------------------
 .../camel/component/seda/SedaEndpoint.java      | 92 --------------------
 .../camel/component/timer/TimerEndpoint.java    | 30 ++-----
 .../camel/component/xslt/XsltEndpoint.java      | 21 ++---
 .../xslt/ManagedXsltOutputBytesTest.java        | 76 ++++++++++++++++
 4 files changed, 87 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/f28399ff/camel-core/src/main/java/org/apache/camel/component/seda/SedaEndpoint.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/seda/SedaEndpoint.java b/camel-core/src/main/java/org/apache/camel/component/seda/SedaEndpoint.java
index 1b82935..7b10d1a 100644
--- a/camel-core/src/main/java/org/apache/camel/component/seda/SedaEndpoint.java
+++ b/camel-core/src/main/java/org/apache/camel/component/seda/SedaEndpoint.java
@@ -27,7 +27,6 @@ import java.util.concurrent.ExecutorService;
 import org.apache.camel.Component;
 import org.apache.camel.Consumer;
 import org.apache.camel.Exchange;
-import org.apache.camel.Message;
 import org.apache.camel.MultipleConsumersSupport;
 import org.apache.camel.PollingConsumer;
 import org.apache.camel.Processor;
@@ -43,8 +42,6 @@ import org.apache.camel.spi.Metadata;
 import org.apache.camel.spi.UriEndpoint;
 import org.apache.camel.spi.UriParam;
 import org.apache.camel.spi.UriPath;
-import org.apache.camel.util.EndpointHelper;
-import org.apache.camel.util.MessageHelper;
 import org.apache.camel.util.ServiceHelper;
 import org.apache.camel.util.URISupport;
 import org.slf4j.Logger;
@@ -405,7 +402,6 @@ public class SedaEndpoint extends DefaultEndpoint implements BrowsableEndpoint,
         this.purgeWhenStopping = purgeWhenStopping;
     }
 
-    @ManagedAttribute(description = "Singleton")
     public boolean isSingleton() {
         return true;
     }
@@ -445,94 +441,6 @@ public class SedaEndpoint extends DefaultEndpoint implements BrowsableEndpoint,
         return new HashSet<SedaProducer>(producers);
     }
 
-    @ManagedOperation(description = "Current number of Exchanges in Queue")
-    public long queueSize() {
-        return getExchanges().size();
-    }
-
-    @ManagedOperation(description = "Get Exchange from queue by index")
-    public String browseExchange(Integer index) {
-        List<Exchange> exchanges = getExchanges();
-        if (index >= exchanges.size()) {
-            return null;
-        }
-        Exchange exchange = exchanges.get(index);
-        if (exchange == null) {
-            return null;
-        }
-        // must use java type with JMX such as java.lang.String
-        return exchange.toString();
-    }
-
-    @ManagedOperation(description = "Get message body from queue by index")
-    public String browseMessageBody(Integer index) {
-        List<Exchange> exchanges = getExchanges();
-        if (index >= exchanges.size()) {
-            return null;
-        }
-        Exchange exchange = exchanges.get(index);
-        if (exchange == null) {
-            return null;
-        }
-
-        // must use java type with JMX such as java.lang.String
-        String body;
-        if (exchange.hasOut()) {
-            body = exchange.getOut().getBody(String.class);
-        } else {
-            body = exchange.getIn().getBody(String.class);
-        }
-
-        return body;
-    }
-
-    @ManagedOperation(description = "Get message as XML from queue by index")
-    public String browseMessageAsXml(Integer index, Boolean includeBody) {
-        List<Exchange> exchanges = getExchanges();
-        if (index >= exchanges.size()) {
-            return null;
-        }
-        Exchange exchange = exchanges.get(index);
-        if (exchange == null) {
-            return null;
-        }
-
-        Message msg = exchange.hasOut() ? exchange.getOut() : exchange.getIn();
-        String xml = MessageHelper.dumpAsXml(msg, includeBody);
-
-        return xml;
-    }
-
-    @ManagedOperation(description = "Gets all the messages as XML from the queue")
-    public String browseAllMessagesAsXml(Boolean includeBody) {
-        return browseRangeMessagesAsXml(0, Integer.MAX_VALUE, includeBody);
-    }
-
-    @ManagedOperation(description = "Gets the range of messages as XML from the queue")
-    public String browseRangeMessagesAsXml(Integer fromIndex, Integer toIndex, Boolean includeBody) {
-        return EndpointHelper.browseRangeMessagesAsXml(this, fromIndex, toIndex, includeBody);
-    }
-
-    @ManagedAttribute(description = "Camel context ID")
-    public String getCamelId() {
-        return getCamelContext().getName();
-    }
-
-    @ManagedAttribute(description = "Camel ManagementName")
-    public String getCamelManagementName() {
-        return getCamelContext().getManagementName();
-    }
-
-    @ManagedAttribute(description = "Endpoint URI", mask = true)
-    public String getEndpointUri() {
-        return super.getEndpointUri();
-    }
-
-    @ManagedAttribute(description = "Endpoint service state")
-    public String getState() {
-        return getStatus().name();
-    }
-
     void onStarted(SedaProducer producer) {
         producers.add(producer);
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/f28399ff/camel-core/src/main/java/org/apache/camel/component/timer/TimerEndpoint.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/timer/TimerEndpoint.java b/camel-core/src/main/java/org/apache/camel/component/timer/TimerEndpoint.java
index 1ddcc81..a3ce56d 100644
--- a/camel-core/src/main/java/org/apache/camel/component/timer/TimerEndpoint.java
+++ b/camel-core/src/main/java/org/apache/camel/component/timer/TimerEndpoint.java
@@ -86,6 +86,11 @@ public class TimerEndpoint extends DefaultEndpoint implements MultipleConsumersS
     }
 
     @Override
+    public boolean isSingleton() {
+        return true;
+    }
+
+    @Override
     protected void doStart() throws Exception {
         super.doStart();
         // do nothing, the timer will be set when the first consumer will request it
@@ -203,31 +208,6 @@ public class TimerEndpoint extends DefaultEndpoint implements MultipleConsumersS
         this.time = time;
     }
 
-    @ManagedAttribute(description = "Singleton")
-    public boolean isSingleton() {
-        return true;
-    }
-
-    @ManagedAttribute(description = "Camel id")
-    public String getCamelId() {
-        return this.getCamelContext().getName();
-    }
-
-    @ManagedAttribute(description = "Camel ManagementName")
-    public String getCamelManagementName() {
-        return this.getCamelContext().getManagementName();
-    }
-
-    @ManagedAttribute(description = "Endpoint Uri")
-    public String getEndpointUri() {
-        return super.getEndpointUri();
-    }
-
-    @ManagedAttribute(description = "Endpoint State")
-    public String getState() {
-        return getStatus().name();
-    }
-
     public Timer getTimer(TimerConsumer consumer) {
         if (timer != null) {
             // use custom timer

http://git-wip-us.apache.org/repos/asf/camel/blob/f28399ff/camel-core/src/main/java/org/apache/camel/component/xslt/XsltEndpoint.java
----------------------------------------------------------------------
diff --git a/camel-core/src/main/java/org/apache/camel/component/xslt/XsltEndpoint.java b/camel-core/src/main/java/org/apache/camel/component/xslt/XsltEndpoint.java
index 8d3db9e..6a4ecd0 100644
--- a/camel-core/src/main/java/org/apache/camel/component/xslt/XsltEndpoint.java
+++ b/camel-core/src/main/java/org/apache/camel/component/xslt/XsltEndpoint.java
@@ -106,21 +106,6 @@ public class XsltEndpoint extends ProcessorEndpoint {
         return contentCache;
     }
 
-    @ManagedAttribute(description = "Endpoint State")
-    public String getState() {
-        return getStatus().name();
-    }
-
-    @ManagedAttribute(description = "Camel ID")
-    public String getCamelId() {
-        return getCamelContext().getName();
-    }
-
-    @ManagedAttribute(description = "Camel ManagementName")
-    public String getCamelManagementName() {
-        return getCamelContext().getManagementName();
-    }
-
     public XsltEndpoint findOrCreateEndpoint(String uri, String newResourceUri) {
         String newUri = uri.replace(resourceUri, newResourceUri);
         LOG.trace("Getting endpoint with URI: {}", newUri);
@@ -151,6 +136,7 @@ public class XsltEndpoint extends ProcessorEndpoint {
         this.xslt = xslt;
     }
 
+    @ManagedAttribute(description = "The name of the template to load from classpath or file system")
     public String getResourceUri() {
         return resourceUri;
     }
@@ -195,6 +181,7 @@ public class XsltEndpoint extends ProcessorEndpoint {
         this.transformerFactory = transformerFactory;
     }
 
+    @ManagedAttribute(description = "Whether to use Saxon as the transformerFactoryClass")
     public boolean isSaxon() {
         return saxon;
     }
@@ -219,6 +206,7 @@ public class XsltEndpoint extends ProcessorEndpoint {
         this.resultHandlerFactory = resultHandlerFactory;
     }
 
+    @ManagedAttribute(description = "Whether or not to throw an exception if the input body is null")
     public boolean isFailOnNullBody() {
         return failOnNullBody;
     }
@@ -230,6 +218,7 @@ public class XsltEndpoint extends ProcessorEndpoint {
         this.failOnNullBody = failOnNullBody;
     }
 
+    @ManagedAttribute(description = "What kind of option to use.")
     public XsltOutput getOutput() {
         return output;
     }
@@ -268,6 +257,7 @@ public class XsltEndpoint extends ProcessorEndpoint {
         this.errorListener = errorListener;
     }
 
+    @ManagedAttribute(description = "Cache for the resource content (the stylesheet file) when it is loaded.")
     public boolean isContentCache() {
         return contentCache;
     }
@@ -292,6 +282,7 @@ public class XsltEndpoint extends ProcessorEndpoint {
         this.uriResolver = uriResolver;
     }
 
+    @ManagedAttribute(description = "Whether to allow using StAX as the javax.xml.transform.Source")
     public boolean isAllowStAX() {
         return allowStAX;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/f28399ff/camel-core/src/test/java/org/apache/camel/component/xslt/ManagedXsltOutputBytesTest.java
----------------------------------------------------------------------
diff --git a/camel-core/src/test/java/org/apache/camel/component/xslt/ManagedXsltOutputBytesTest.java b/camel-core/src/test/java/org/apache/camel/component/xslt/ManagedXsltOutputBytesTest.java
new file mode 100644
index 0000000..a0c1ae7
--- /dev/null
+++ b/camel-core/src/test/java/org/apache/camel/component/xslt/ManagedXsltOutputBytesTest.java
@@ -0,0 +1,76 @@
+/**
+ * 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.component.xslt;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
+
+import org.apache.camel.ContextTestSupport;
+import org.apache.camel.builder.RouteBuilder;
+import org.apache.camel.component.mock.MockEndpoint;
+
+/**
+ *
+ */
+public class ManagedXsltOutputBytesTest extends ContextTestSupport {
+
+    @Override
+    protected boolean useJmx() {
+        return true;
+    }
+
+    protected MBeanServer getMBeanServer() {
+        return context.getManagementStrategy().getManagementAgent().getMBeanServer();
+    }
+
+    public void testXsltOutput() throws Exception {
+        MockEndpoint mock = getMockEndpoint("mock:result");
+        mock.expectedBodiesReceived("<?xml version=\"1.0\" encoding=\"UTF-8\"?><goodbye>world!</goodbye>");
+        mock.message(0).body().isInstanceOf(byte[].class);
+
+        template.sendBody("direct:start", "<hello>world!</hello>");
+
+        assertMockEndpointsSatisfied();
+
+        MBeanServer mbeanServer = getMBeanServer();
+
+        ObjectName on = ObjectName.getInstance("org.apache.camel:context=camel-1,type=endpoints,name=\"xslt://org/apache/camel/component/xslt/example.xsl\\?output=bytes\"");
+        String uri = (String) mbeanServer.getAttribute(on, "EndpointUri");
+        assertEquals("xslt://org/apache/camel/component/xslt/example.xsl?output=bytes", uri);
+
+        Boolean saxon = (Boolean) mbeanServer.getAttribute(on, "Saxon");
+        assertEquals(false, saxon.booleanValue());
+
+        XsltOutput output = (XsltOutput) mbeanServer.getAttribute(on, "Output");
+        assertEquals(XsltOutput.bytes, output);
+
+        String state = (String) mbeanServer.getAttribute(on, "State");
+        assertEquals("Started", state);
+    }
+
+    @Override
+    protected RouteBuilder createRouteBuilder() throws Exception {
+        return new RouteBuilder() {
+            @Override
+            public void configure() throws Exception {
+                from("direct:start")
+                    .to("xslt:org/apache/camel/component/xslt/example.xsl?output=bytes")
+                    .to("mock:result");
+            }
+        };
+    }
+}


[5/6] camel git commit: CAMEL-8147: Camel JMX when using custom @ManagedResource should mixin the standard mbean operations/attributes so you get both worlds.

Posted by da...@apache.org.
CAMEL-8147: Camel JMX when using custom @ManagedResource should mixin the standard mbean operations/attributes so you get both worlds.


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/de71c172
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/de71c172
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/de71c172

Branch: refs/heads/master
Commit: de71c172523e8b100aa33d139873641e3f6b7271
Parents: 0a6be48
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Aug 11 14:40:03 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Tue Aug 11 15:30:41 2015 +0200

----------------------------------------------------------------------
 .../jms/JmsEndpointConfigurationTest.java       |  2 +-
 .../twitter/TwitterEndpointPolling.java         | 29 +-------------
 .../component/zookeeper/ZooKeeperEndpoint.java  | 41 --------------------
 .../zookeeper/ZooKeeperManagedEndpointTest.java | 19 ++++-----
 4 files changed, 9 insertions(+), 82 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/de71c172/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsEndpointConfigurationTest.java
----------------------------------------------------------------------
diff --git a/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsEndpointConfigurationTest.java b/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsEndpointConfigurationTest.java
index e4d0d7b..b504e62 100644
--- a/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsEndpointConfigurationTest.java
+++ b/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsEndpointConfigurationTest.java
@@ -310,7 +310,7 @@ public class JmsEndpointConfigurationTest extends CamelTestSupport {
         assertEquals(1000L, endpoint.getRequestTimeoutCheckerInterval());
         assertEquals(0, endpoint.getRunningMessageListeners());
         assertNull(endpoint.getSelector());
-        assertEquals(ServiceStatus.Started.toString(), endpoint.getStatus());
+        assertEquals(ServiceStatus.Started, endpoint.getStatus());
         assertEquals(-1, endpoint.getTimeToLive());
         assertNull(endpoint.getTransactionName());
         assertEquals(-1, endpoint.getTransactionTimeout());

http://git-wip-us.apache.org/repos/asf/camel/blob/de71c172/components/camel-twitter/src/main/java/org/apache/camel/component/twitter/TwitterEndpointPolling.java
----------------------------------------------------------------------
diff --git a/components/camel-twitter/src/main/java/org/apache/camel/component/twitter/TwitterEndpointPolling.java b/components/camel-twitter/src/main/java/org/apache/camel/component/twitter/TwitterEndpointPolling.java
index e1e8309..2d42b93 100644
--- a/components/camel-twitter/src/main/java/org/apache/camel/component/twitter/TwitterEndpointPolling.java
+++ b/components/camel-twitter/src/main/java/org/apache/camel/component/twitter/TwitterEndpointPolling.java
@@ -19,7 +19,6 @@ package org.apache.camel.component.twitter;
 import org.apache.camel.Consumer;
 import org.apache.camel.Processor;
 import org.apache.camel.Producer;
-import org.apache.camel.ServiceStatus;
 import org.apache.camel.api.management.ManagedAttribute;
 import org.apache.camel.api.management.ManagedResource;
 import org.apache.camel.component.twitter.consumer.Twitter4JConsumer;
@@ -59,7 +58,7 @@ public class TwitterEndpointPolling extends DefaultPollingEndpoint implements Tw
         return Twitter4JFactory.getProducer(this, getEndpointUri());
     }
 
-    @ManagedAttribute
+    @Override
     public boolean isSingleton() {
         return true;
     }
@@ -72,32 +71,6 @@ public class TwitterEndpointPolling extends DefaultPollingEndpoint implements Tw
         this.properties = properties;
     }
 
-    @ManagedAttribute(description = "Camel ID")
-    public String getCamelId() {
-        return getCamelContext().getName();
-    }
-
-    @ManagedAttribute(description = "Camel ManagementName")
-    public String getCamelManagementName() {
-        return getCamelContext().getManagementName();
-    }
-
-    @ManagedAttribute(description = "Endpoint Uri", mask = true)
-    @Override
-    public String getEndpointUri() {
-        return super.getEndpointUri();
-    }
-
-    @ManagedAttribute(description = "Service State")
-    public String getState() {
-        ServiceStatus status = this.getStatus();
-        // if no status exists then its stopped
-        if (status == null) {
-            status = ServiceStatus.Stopped;
-        }
-        return status.name();
-    }
-
     @ManagedAttribute
     public String getKeywords() {
         return getProperties().getKeywords();

http://git-wip-us.apache.org/repos/asf/camel/blob/de71c172/components/camel-zookeeper/src/main/java/org/apache/camel/component/zookeeper/ZooKeeperEndpoint.java
----------------------------------------------------------------------
diff --git a/components/camel-zookeeper/src/main/java/org/apache/camel/component/zookeeper/ZooKeeperEndpoint.java b/components/camel-zookeeper/src/main/java/org/apache/camel/component/zookeeper/ZooKeeperEndpoint.java
index 64272ca..0cc2036 100644
--- a/components/camel-zookeeper/src/main/java/org/apache/camel/component/zookeeper/ZooKeeperEndpoint.java
+++ b/components/camel-zookeeper/src/main/java/org/apache/camel/component/zookeeper/ZooKeeperEndpoint.java
@@ -21,7 +21,6 @@ import java.util.List;
 import org.apache.camel.Consumer;
 import org.apache.camel.Processor;
 import org.apache.camel.Producer;
-import org.apache.camel.ServiceStatus;
 import org.apache.camel.api.management.ManagedAttribute;
 import org.apache.camel.api.management.ManagedOperation;
 import org.apache.camel.api.management.ManagedResource;
@@ -71,33 +70,6 @@ public class ZooKeeperEndpoint extends DefaultEndpoint {
         return connectionManager;
     }
 
-    @ManagedAttribute(description = "Camel ID")
-    public String getCamelId() {
-        return getCamelContext().getName();
-    }
-
-    @ManagedAttribute(description = "Camel ManagementName")
-    public String getCamelManagementName() {
-        return getCamelContext().getManagementName();
-    }
-
-    @ManagedAttribute(description = "Endpoint Uri", mask = true)
-    @Override
-    public String getEndpointUri() {
-        return super.getEndpointUri();
-    }
-
-    @ManagedAttribute(description = "Service State")
-    public String getState() {
-        ServiceStatus status = this.getStatus();
-        // if no status exists then its stopped
-        if (status == null) {
-            status = ServiceStatus.Stopped;
-        }
-        return status.name();
-    }
-
-    @ManagedAttribute
     public void setPath(String path) {
         getConfiguration().setPath(path);
     }
@@ -112,7 +84,6 @@ public class ZooKeeperEndpoint extends DefaultEndpoint {
         return getConfiguration().getTimeout();
     }
 
-    @ManagedAttribute
     public void setTimeout(int timeout) {
         getConfiguration().setTimeout(timeout);
     }
@@ -122,7 +93,6 @@ public class ZooKeeperEndpoint extends DefaultEndpoint {
         return getConfiguration().isRepeat();
     }
 
-    @ManagedAttribute
     public void setRepeat(boolean shouldRepeat) {
         getConfiguration().setRepeat(shouldRepeat);
     }
@@ -132,7 +102,6 @@ public class ZooKeeperEndpoint extends DefaultEndpoint {
         return getConfiguration().getServers();
     }
 
-    @ManagedAttribute
     public void setServers(List<String> servers) {
         getConfiguration().setServers(servers);
     }
@@ -142,7 +111,6 @@ public class ZooKeeperEndpoint extends DefaultEndpoint {
         return getConfiguration().isListChildren();
     }
 
-    @ManagedAttribute
     public void setListChildren(boolean listChildren) {
         getConfiguration().setListChildren(listChildren);
     }
@@ -152,7 +120,6 @@ public class ZooKeeperEndpoint extends DefaultEndpoint {
         return getConfiguration().isCreate();
     }
 
-    @ManagedAttribute
     public void setCreate(boolean shouldCreate) {
         getConfiguration().setCreate(shouldCreate);
     }
@@ -162,7 +129,6 @@ public class ZooKeeperEndpoint extends DefaultEndpoint {
         return getConfiguration().getBackoff();
     }
 
-    @ManagedAttribute
     public void setBackoff(long backoff) {
         getConfiguration().setBackoff(backoff);
     }
@@ -171,7 +137,6 @@ public class ZooKeeperEndpoint extends DefaultEndpoint {
      * @deprecated The usage of this property has no effect at all.
      */
     @Deprecated
-    @ManagedAttribute
     public boolean getAwaitExistence() {
         return getConfiguration().shouldAwaitExistence();
     }
@@ -180,7 +145,6 @@ public class ZooKeeperEndpoint extends DefaultEndpoint {
      * @deprecated The usage of this property has no effect at all.
      */
     @Deprecated
-    @ManagedAttribute
     public void setAwaitExistence(boolean awaitExistence) {
         getConfiguration().setAwaitExistence(awaitExistence);
     }
@@ -195,16 +159,11 @@ public class ZooKeeperEndpoint extends DefaultEndpoint {
         getConfiguration().getServers().clear();
     }
 
-    public Object getManagedObject(ZooKeeperEndpoint arg0) {
-        return this;
-    }
-
     @ManagedAttribute
     public boolean isSendEmptyMessageOnDelete() {
         return getConfiguration().isSendEmptyMessageOnDelete();
     }
 
-    @ManagedAttribute
     public void setSendEmptyMessageOnDelete(boolean sendEmptyMessageOnDelete) {
         getConfiguration().setSendEmptyMessageOnDelete(sendEmptyMessageOnDelete);
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/de71c172/components/camel-zookeeper/src/test/java/org/apache/camel/component/zookeeper/ZooKeeperManagedEndpointTest.java
----------------------------------------------------------------------
diff --git a/components/camel-zookeeper/src/test/java/org/apache/camel/component/zookeeper/ZooKeeperManagedEndpointTest.java b/components/camel-zookeeper/src/test/java/org/apache/camel/component/zookeeper/ZooKeeperManagedEndpointTest.java
index ae67866..c8c9e5b 100644
--- a/components/camel-zookeeper/src/test/java/org/apache/camel/component/zookeeper/ZooKeeperManagedEndpointTest.java
+++ b/components/camel-zookeeper/src/test/java/org/apache/camel/component/zookeeper/ZooKeeperManagedEndpointTest.java
@@ -64,13 +64,12 @@ public class ZooKeeperManagedEndpointTest extends CamelTestSupport {
         assertEquals("Could not find zookeper endpoint: " + s, 1, s.size());
         ObjectName zepName = new ArrayList<ObjectName>(s).get(0);
 
-        verifyManagedAttribute(zepName, "Path", "/someotherpath");
-        verifyManagedAttribute(zepName, "Create", true);
-        verifyManagedAttribute(zepName, "Repeat", true);
-        verifyManagedAttribute(zepName, "ListChildren", true);
-        verifyManagedAttribute(zepName, "AwaitExistence", true);
-        verifyManagedAttribute(zepName, "Timeout", 12345);
-        verifyManagedAttribute(zepName, "Backoff", 12345L);
+        verifyManagedAttribute(zepName, "Path", "/node");
+        verifyManagedAttribute(zepName, "Create", false);
+        verifyManagedAttribute(zepName, "Repeat", false);
+        verifyManagedAttribute(zepName, "ListChildren", false);
+        verifyManagedAttribute(zepName, "Timeout", 1000);
+        verifyManagedAttribute(zepName, "Backoff", 2000L);
 
         getMBeanServer().invoke(zepName, "clearServers", null, JmxUtils.getMethodSignature(ZooKeeperEndpoint.class.getMethod("clearServers", null)));
         getMBeanServer().invoke(zepName, "addServer", new Object[]{"someserver:12345"},
@@ -78,29 +77,25 @@ public class ZooKeeperManagedEndpointTest extends CamelTestSupport {
     }
 
     private void verifyManagedAttribute(ObjectName zepName, String attributeName, String attributeValue) throws Exception {
-        getMBeanServer().setAttribute(zepName, new Attribute(attributeName, attributeValue));
         assertEquals(attributeValue, getMBeanServer().getAttribute(zepName, attributeName));
     }
 
     private void verifyManagedAttribute(ObjectName zepName, String attributeName, Integer attributeValue) throws Exception {
-        getMBeanServer().setAttribute(zepName, new Attribute(attributeName, attributeValue));
         assertEquals(attributeValue, getMBeanServer().getAttribute(zepName, attributeName));
     }
 
     private void verifyManagedAttribute(ObjectName zepName, String attributeName, Boolean attributeValue) throws Exception {
-        getMBeanServer().setAttribute(zepName, new Attribute(attributeName, attributeValue));
         assertEquals(attributeValue, getMBeanServer().getAttribute(zepName, attributeName));
     }
 
     private void verifyManagedAttribute(ObjectName zepName, String attributeName, Long attributeValue) throws Exception {
-        getMBeanServer().setAttribute(zepName, new Attribute(attributeName, attributeValue));
         assertEquals(attributeValue, getMBeanServer().getAttribute(zepName, attributeName));
     }
 
     protected RouteBuilder createRouteBuilder() {
         return new RouteBuilder() {
             public void configure() {
-                from("zookeeper://localhost:" + ZooKeeperTestSupport.getServerPort() + "/node").to("mock:test");
+                from("zookeeper://localhost:" + ZooKeeperTestSupport.getServerPort() + "/node?timeout=1000&backoff=2000").to("mock:test");
             }
         };
     }


[2/6] camel git commit: CAMEL-8147: Camel JMX when using custom @ManagedResource should mixin the standard mbean operations/attributes so you get both worlds.

Posted by da...@apache.org.
CAMEL-8147: Camel JMX when using custom @ManagedResource should mixin the standard mbean operations/attributes so you get both worlds.


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/0a6be48e
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/0a6be48e
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/0a6be48e

Branch: refs/heads/master
Commit: 0a6be48e46a1aac779e4c40c7257a680986a2065
Parents: c35c0e9
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Aug 11 14:36:06 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Tue Aug 11 15:30:40 2015 +0200

----------------------------------------------------------------------
 .../camel/component/jms/JmsQueueEndpoint.java   | 72 --------------------
 .../camel/component/jms/BrowsableQueueTest.java |  6 +-
 2 files changed, 3 insertions(+), 75 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/0a6be48e/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsQueueEndpoint.java
----------------------------------------------------------------------
diff --git a/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsQueueEndpoint.java b/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsQueueEndpoint.java
index d415cb9..389e75f 100644
--- a/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsQueueEndpoint.java
+++ b/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsQueueEndpoint.java
@@ -22,13 +22,9 @@ import javax.jms.JMSException;
 import javax.jms.Queue;
 
 import org.apache.camel.Exchange;
-import org.apache.camel.Message;
 import org.apache.camel.api.management.ManagedAttribute;
-import org.apache.camel.api.management.ManagedOperation;
 import org.apache.camel.api.management.ManagedResource;
 import org.apache.camel.spi.BrowsableEndpoint;
-import org.apache.camel.util.EndpointHelper;
-import org.apache.camel.util.MessageHelper;
 import org.springframework.jms.core.JmsOperations;
 
 /**
@@ -103,74 +99,6 @@ public class JmsQueueEndpoint extends JmsEndpoint implements BrowsableEndpoint {
         return queueBrowseStrategy.browse(template, queue, this);
     }
 
-    @ManagedOperation(description = "Current number of Exchanges in Queue")
-    public long queueSize() {
-        return getExchanges().size();
-    }
-
-    @ManagedOperation(description = "Get Exchange from queue by index")
-    public String browseExchange(Integer index) {
-        List<Exchange> exchanges = getExchanges();
-        if (index >= exchanges.size()) {
-            return null;
-        }
-        Exchange exchange = exchanges.get(index);
-        if (exchange == null) {
-            return null;
-        }
-        // must use java type with JMX such as java.lang.String
-        return exchange.toString();
-    }
-
-    @ManagedOperation(description = "Get message body from queue by index")
-    public String browseMessageBody(Integer index) {
-        List<Exchange> exchanges = getExchanges();
-        if (index >= exchanges.size()) {
-            return null;
-        }
-        Exchange exchange = exchanges.get(index);
-        if (exchange == null) {
-            return null;
-        }
-
-        // must use java type with JMX such as java.lang.String
-        String body;
-        if (exchange.hasOut()) {
-            body = exchange.getOut().getBody(String.class);
-        } else {
-            body = exchange.getIn().getBody(String.class);
-        }
-
-        return body;
-    }
-
-    @ManagedOperation(description = "Get message as XML from queue by index")
-    public String browseMessageAsXml(Integer index, Boolean includeBody) {
-        List<Exchange> exchanges = getExchanges();
-        if (index >= exchanges.size()) {
-            return null;
-        }
-        Exchange exchange = exchanges.get(index);
-        if (exchange == null) {
-            return null;
-        }
-
-        Message msg = exchange.hasOut() ? exchange.getOut() : exchange.getIn();
-        String xml = MessageHelper.dumpAsXml(msg, includeBody);
-
-        return xml;
-    }
-
-    @ManagedOperation(description = "Gets all the messages as XML from the queue")
-    public String browseAllMessagesAsXml(Boolean includeBody) {
-        return browseRangeMessagesAsXml(0, Integer.MAX_VALUE, includeBody);
-    }
-
-    @ManagedOperation(description = "Gets the range of messages as XML from the queue")
-    public String browseRangeMessagesAsXml(Integer fromIndex, Integer toIndex, Boolean includeBody) {
-        return EndpointHelper.browseRangeMessagesAsXml(this, fromIndex, toIndex, includeBody);
-    }
-
     protected QueueBrowseStrategy createQueueBrowseStrategy() {
         return new DefaultQueueBrowseStrategy();
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/0a6be48e/components/camel-jms/src/test/java/org/apache/camel/component/jms/BrowsableQueueTest.java
----------------------------------------------------------------------
diff --git a/components/camel-jms/src/test/java/org/apache/camel/component/jms/BrowsableQueueTest.java b/components/camel-jms/src/test/java/org/apache/camel/component/jms/BrowsableQueueTest.java
index f0080c8..b30314f 100644
--- a/components/camel-jms/src/test/java/org/apache/camel/component/jms/BrowsableQueueTest.java
+++ b/components/camel-jms/src/test/java/org/apache/camel/component/jms/BrowsableQueueTest.java
@@ -52,7 +52,7 @@ public class BrowsableQueueTest extends CamelTestSupport {
         assertEquals(6, endpoint.getMaximumBrowseSize());
         List<Exchange> list = endpoint.getExchanges();
         LOG.debug("Received: " + list);
-        assertEquals("Size of list", 6, endpoint.queueSize());
+        assertEquals("Size of list", 6, endpoint.getExchanges().size());
 
         int index = -1;
         for (Exchange exchange : list) {
@@ -77,7 +77,7 @@ public class BrowsableQueueTest extends CamelTestSupport {
         assertEquals(10, endpoint.getMaximumBrowseSize());
         List<Exchange> list = endpoint.getExchanges();
         LOG.debug("Received: " + list);
-        assertEquals("Size of list", 8, endpoint.queueSize());
+        assertEquals("Size of list", 8, endpoint.getExchanges().size());
 
         int index = -1;
         for (Exchange exchange : list) {
@@ -102,7 +102,7 @@ public class BrowsableQueueTest extends CamelTestSupport {
         assertEquals(-1, endpoint.getMaximumBrowseSize());
         List<Exchange> list = endpoint.getExchanges();
         LOG.debug("Received: " + list);
-        assertEquals("Size of list", 8, endpoint.queueSize());
+        assertEquals("Size of list", 8, endpoint.getExchanges().size());
 
         int index = -1;
         for (Exchange exchange : list) {


[3/6] camel git commit: CAMEL-8147: Camel JMX when using custom @ManagedResource should mixin the standard mbean operations/attributes so you get both worlds.

Posted by da...@apache.org.
CAMEL-8147: Camel JMX when using custom @ManagedResource should mixin the standard mbean operations/attributes so you get both worlds.


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/c35c0e94
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/c35c0e94
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/c35c0e94

Branch: refs/heads/master
Commit: c35c0e948ad35b65ab846b3ef73083d69c15a7d1
Parents: f28399f
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Aug 11 14:33:09 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Tue Aug 11 15:30:40 2015 +0200

----------------------------------------------------------------------
 .../component/disruptor/DisruptorEndpoint.java  | 43 +++++---------------
 .../apache/camel/component/jms/JmsEndpoint.java | 29 -------------
 .../jms/JmsEndpointConfigurationTest.java       |  4 +-
 .../component/jms/ManagedJmsEndpointTest.java   |  3 ++
 4 files changed, 15 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/c35c0e94/components/camel-disruptor/src/main/java/org/apache/camel/component/disruptor/DisruptorEndpoint.java
----------------------------------------------------------------------
diff --git a/components/camel-disruptor/src/main/java/org/apache/camel/component/disruptor/DisruptorEndpoint.java b/components/camel-disruptor/src/main/java/org/apache/camel/component/disruptor/DisruptorEndpoint.java
index ef83f41..33debc7 100644
--- a/components/camel-disruptor/src/main/java/org/apache/camel/component/disruptor/DisruptorEndpoint.java
+++ b/components/camel-disruptor/src/main/java/org/apache/camel/component/disruptor/DisruptorEndpoint.java
@@ -85,31 +85,6 @@ public class DisruptorEndpoint extends DefaultEndpoint implements MultipleConsum
         this.blockWhenFull = blockWhenFull;
     }
 
-    @ManagedAttribute(description = "Camel ID")
-    public String getCamelId() {
-        return getCamelContext().getName();
-    }
-
-    @ManagedAttribute(description = "Camel ManagementName")
-    public String getCamelManagementName() {
-        return getCamelContext().getManagementName();
-    }
-
-    @ManagedAttribute(description = "Endpoint Uri", mask = true)
-    public String getEndpointUri() {
-        return super.getEndpointUri();
-    }
-
-    @ManagedAttribute(description = "Service State")
-    public String getState() {
-        ServiceStatus status = this.getStatus();
-        // if no status exists then its stopped
-        if (status == null) {
-            status = ServiceStatus.Stopped;
-        }
-        return status.name();
-    }
-
     @ManagedAttribute(description = "Queue name")
     public String getName() {
         return name;
@@ -138,6 +113,7 @@ public class DisruptorEndpoint extends DefaultEndpoint implements MultipleConsum
         return concurrentConsumers;
     }
 
+    @ManagedAttribute(description = "Option to specify whether the caller should wait for the async task to complete or not before continuing")
     public WaitForTaskToComplete getWaitForTaskToComplete() {
         return waitForTaskToComplete;
     }
@@ -151,7 +127,7 @@ public class DisruptorEndpoint extends DefaultEndpoint implements MultipleConsum
         this.waitForTaskToComplete = waitForTaskToComplete;
     }
 
-    @ManagedAttribute
+    @ManagedAttribute(description = "Timeout (in milliseconds) before a producer will stop waiting for an asynchronous task to complete")
     public long getTimeout() {
         return timeout;
     }
@@ -164,13 +140,18 @@ public class DisruptorEndpoint extends DefaultEndpoint implements MultipleConsum
         this.timeout = timeout;
     }
 
+    @Override
+    @ManagedAttribute(description = "Specifies whether multiple consumers are allowed")
+    public boolean isMultipleConsumersSupported() {
+        return isMultipleConsumers();
+    }
+
     /**
      * Specifies whether multiple consumers are allowed.
      * If enabled, you can use Disruptor for Publish-Subscribe messaging.
      * That is, you can send a message to the queue and have each consumer receive a copy of the message.
      * When enabled, this option should be specified on every consumer endpoint.
      */
-    @ManagedAttribute
     public boolean isMultipleConsumers() {
         return multipleConsumers;
     }
@@ -189,12 +170,6 @@ public class DisruptorEndpoint extends DefaultEndpoint implements MultipleConsum
         return Collections.unmodifiableSet(producers);
     }
 
-    @Override
-    @ManagedAttribute
-    public boolean isMultipleConsumersSupported() {
-        return isMultipleConsumers();
-    }
-
     @ManagedAttribute
     public boolean isBlockWhenFull() {
         return blockWhenFull;
@@ -209,6 +184,7 @@ public class DisruptorEndpoint extends DefaultEndpoint implements MultipleConsum
         this.blockWhenFull = blockWhenFull;
     }
 
+    @ManagedAttribute(description = "Defines the strategy used by consumer threads to wait on new exchanges to be published")
     public DisruptorWaitStrategy getWaitStrategy() {
         return waitStrategy;
     }
@@ -221,6 +197,7 @@ public class DisruptorEndpoint extends DefaultEndpoint implements MultipleConsum
         this.waitStrategy = waitStrategy;
     }
 
+    @ManagedAttribute(description = " Defines the producers allowed on the Disruptor")
     public DisruptorProducerType getProducerType() {
         return producerType;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/c35c0e94/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsEndpoint.java
----------------------------------------------------------------------
diff --git a/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsEndpoint.java b/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsEndpoint.java
index 3edbbd1..aa7f530 100644
--- a/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsEndpoint.java
+++ b/components/camel-jms/src/main/java/org/apache/camel/component/jms/JmsEndpoint.java
@@ -38,11 +38,9 @@ import org.apache.camel.PollingConsumer;
 import org.apache.camel.Processor;
 import org.apache.camel.Producer;
 import org.apache.camel.Service;
-import org.apache.camel.ServiceStatus;
 import org.apache.camel.api.management.ManagedAttribute;
 import org.apache.camel.api.management.ManagedResource;
 import org.apache.camel.impl.DefaultEndpoint;
-import org.apache.camel.impl.DefaultExchange;
 import org.apache.camel.impl.SynchronousDelegateProducer;
 import org.apache.camel.spi.HeaderFilterStrategy;
 import org.apache.camel.spi.HeaderFilterStrategyAware;
@@ -418,7 +416,6 @@ public class JmsEndpoint extends DefaultEndpoint implements HeaderFilterStrategy
         this.selector = selector;
     }
 
-    @ManagedAttribute
     public boolean isSingleton() {
         return true;
     }
@@ -1217,32 +1214,6 @@ public class JmsEndpoint extends DefaultEndpoint implements HeaderFilterStrategy
         configuration.setReplyToType(type);
     }
 
-    @ManagedAttribute(description = "Camel ID")
-    public String getCamelId() {
-        return getCamelContext().getName();
-    }
-
-    @ManagedAttribute(description = "Camel ManagementName")
-    public String getCamelManagementName() {
-        return getCamelContext().getManagementName();
-    }
-
-    @ManagedAttribute(description = "Endpoint Uri", mask = true)
-    @Override
-    public String getEndpointUri() {
-        return super.getEndpointUri();
-    }
-
-    @ManagedAttribute(description = "Service State")
-    public String getState() {
-        ServiceStatus status = this.getStatus();
-        // if no status exists then its stopped
-        if (status == null) {
-            status = ServiceStatus.Stopped;
-        }
-        return status.name();
-    }
-
     @ManagedAttribute(description = "Number of running message listeners")
     public int getRunningMessageListeners() {
         return runningMessageListeners.get();

http://git-wip-us.apache.org/repos/asf/camel/blob/c35c0e94/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsEndpointConfigurationTest.java
----------------------------------------------------------------------
diff --git a/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsEndpointConfigurationTest.java b/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsEndpointConfigurationTest.java
index 744979e..e4d0d7b 100644
--- a/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsEndpointConfigurationTest.java
+++ b/components/camel-jms/src/test/java/org/apache/camel/component/jms/JmsEndpointConfigurationTest.java
@@ -275,7 +275,7 @@ public class JmsEndpointConfigurationTest extends CamelTestSupport {
         assertNull(endpoint.getAcknowledgementModeName());
         assertEquals(-1, endpoint.getCacheLevel());
         assertNull(endpoint.getCacheLevelName());
-        assertNotNull(endpoint.getCamelId());
+        assertNotNull(endpoint.getCamelContext().getName());
         assertNull(endpoint.getClientId());
         assertNotNull(endpoint.getConnectionFactory());
         assertEquals(1, endpoint.getConcurrentConsumers());
@@ -310,7 +310,7 @@ public class JmsEndpointConfigurationTest extends CamelTestSupport {
         assertEquals(1000L, endpoint.getRequestTimeoutCheckerInterval());
         assertEquals(0, endpoint.getRunningMessageListeners());
         assertNull(endpoint.getSelector());
-        assertEquals(ServiceStatus.Started.toString(), endpoint.getState());
+        assertEquals(ServiceStatus.Started.toString(), endpoint.getStatus());
         assertEquals(-1, endpoint.getTimeToLive());
         assertNull(endpoint.getTransactionName());
         assertEquals(-1, endpoint.getTransactionTimeout());

http://git-wip-us.apache.org/repos/asf/camel/blob/c35c0e94/components/camel-jms/src/test/java/org/apache/camel/component/jms/ManagedJmsEndpointTest.java
----------------------------------------------------------------------
diff --git a/components/camel-jms/src/test/java/org/apache/camel/component/jms/ManagedJmsEndpointTest.java b/components/camel-jms/src/test/java/org/apache/camel/component/jms/ManagedJmsEndpointTest.java
index c06e620..0ce6cff 100644
--- a/components/camel-jms/src/test/java/org/apache/camel/component/jms/ManagedJmsEndpointTest.java
+++ b/components/camel-jms/src/test/java/org/apache/camel/component/jms/ManagedJmsEndpointTest.java
@@ -83,6 +83,9 @@ public class ManagedJmsEndpointTest extends CamelTestSupport {
 
         size = (Long) mbeanServer.invoke(name, "queueSize", null, null);
         assertEquals(1, size.intValue());
+
+        String body = (String) mbeanServer.invoke(name, "browseMessageBody", new Object[]{0}, new String[]{"java.lang.Integer"});
+        assertEquals("Hi World", body);
     }
 
     @Override


[6/6] camel git commit: CAMEL-5402: Camel proxy allows to bind to method interface using @Body @Header and @ExchangeProperty to bind arguments to the exchange

Posted by da...@apache.org.
CAMEL-5402: Camel proxy allows to bind to method interface using @Body @Header and @ExchangeProperty to bind arguments to the exchange


Project: http://git-wip-us.apache.org/repos/asf/camel/repo
Commit: http://git-wip-us.apache.org/repos/asf/camel/commit/dd8ea2f6
Tree: http://git-wip-us.apache.org/repos/asf/camel/tree/dd8ea2f6
Diff: http://git-wip-us.apache.org/repos/asf/camel/diff/dd8ea2f6

Branch: refs/heads/master
Commit: dd8ea2f68836fcd0e5bf48b05b7ecd6caf79d1c7
Parents: de71c17
Author: Claus Ibsen <da...@apache.org>
Authored: Tue Aug 11 15:30:29 2015 +0200
Committer: Claus Ibsen <da...@apache.org>
Committed: Tue Aug 11 15:30:41 2015 +0200

----------------------------------------------------------------------
 .../apache/camel/component/jms/remoting/JmsProxyMEPTest.java   | 6 ++----
 .../org/apache/camel/component/jms/remoting/JmsProxyTest.java  | 6 ++----
 .../org/apache/camel/component/jms/remoting/spring.xml         | 2 +-
 3 files changed, 5 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/camel/blob/dd8ea2f6/components/camel-jms/src/test/java/org/apache/camel/component/jms/remoting/JmsProxyMEPTest.java
----------------------------------------------------------------------
diff --git a/components/camel-jms/src/test/java/org/apache/camel/component/jms/remoting/JmsProxyMEPTest.java b/components/camel-jms/src/test/java/org/apache/camel/component/jms/remoting/JmsProxyMEPTest.java
index 86a076e..d4f01ac 100644
--- a/components/camel-jms/src/test/java/org/apache/camel/component/jms/remoting/JmsProxyMEPTest.java
+++ b/components/camel-jms/src/test/java/org/apache/camel/component/jms/remoting/JmsProxyMEPTest.java
@@ -19,9 +19,8 @@ package org.apache.camel.component.jms.remoting;
 import javax.jms.ConnectionFactory;
 
 import org.apache.camel.CamelContext;
-import org.apache.camel.Endpoint;
+import org.apache.camel.builder.ProxyBuilder;
 import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.component.bean.ProxyHelper;
 import org.apache.camel.component.jms.CamelJmsTestHelper;
 import org.apache.camel.test.junit4.CamelTestSupport;
 import org.junit.Test;
@@ -40,8 +39,7 @@ public class JmsProxyMEPTest extends CamelTestSupport {
         camelContext.addComponent(componentName, jmsComponentAutoAcknowledge(connectionFactory));
 
         // create proxy
-        Endpoint endpoint = camelContext.getEndpoint("direct:calculatorProxy");
-        calculatorProxy = ProxyHelper.createProxy(endpoint, Calculator.class);
+        calculatorProxy = new ProxyBuilder(camelContext).binding(false).endpoint("direct:calculatorProxy").build(Calculator.class);
 
         return camelContext;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/dd8ea2f6/components/camel-jms/src/test/java/org/apache/camel/component/jms/remoting/JmsProxyTest.java
----------------------------------------------------------------------
diff --git a/components/camel-jms/src/test/java/org/apache/camel/component/jms/remoting/JmsProxyTest.java b/components/camel-jms/src/test/java/org/apache/camel/component/jms/remoting/JmsProxyTest.java
index 644e64a..7aa60dd 100644
--- a/components/camel-jms/src/test/java/org/apache/camel/component/jms/remoting/JmsProxyTest.java
+++ b/components/camel-jms/src/test/java/org/apache/camel/component/jms/remoting/JmsProxyTest.java
@@ -19,9 +19,8 @@ package org.apache.camel.component.jms.remoting;
 import javax.jms.ConnectionFactory;
 
 import org.apache.camel.CamelContext;
-import org.apache.camel.Endpoint;
+import org.apache.camel.builder.ProxyBuilder;
 import org.apache.camel.builder.RouteBuilder;
-import org.apache.camel.component.bean.ProxyHelper;
 import org.apache.camel.component.jms.CamelJmsTestHelper;
 import org.apache.camel.test.junit4.CamelTestSupport;
 import org.junit.Test;
@@ -40,8 +39,7 @@ public class JmsProxyTest extends CamelTestSupport {
         camelContext.addComponent(componentName, jmsComponentAutoAcknowledge(connectionFactory));
 
         // create proxy
-        Endpoint endpoint = camelContext.getEndpoint("direct:calculatorProxy");
-        calculatorProxy = ProxyHelper.createProxy(endpoint, Calculator.class);
+        calculatorProxy = new ProxyBuilder(camelContext).binding(false).endpoint("direct:calculatorProxy").build(Calculator.class);
 
         return camelContext;
     }

http://git-wip-us.apache.org/repos/asf/camel/blob/dd8ea2f6/components/camel-jms/src/test/resources/org/apache/camel/component/jms/remoting/spring.xml
----------------------------------------------------------------------
diff --git a/components/camel-jms/src/test/resources/org/apache/camel/component/jms/remoting/spring.xml b/components/camel-jms/src/test/resources/org/apache/camel/component/jms/remoting/spring.xml
index 6aa7b9b..5675da4 100644
--- a/components/camel-jms/src/test/resources/org/apache/camel/component/jms/remoting/spring.xml
+++ b/components/camel-jms/src/test/resources/org/apache/camel/component/jms/remoting/spring.xml
@@ -28,7 +28,7 @@
   <camelContext id="camel" xmlns="http://camel.apache.org/schema/spring">
       <jmxAgent id="agent" disabled="true"/>
     <!-- create the client proxy -->
-    <proxy id="sayProxy" serviceUrl="activemq:test.serviceQueue"
+    <proxy id="sayProxy" binding="false" serviceUrl="activemq:test.serviceQueue"
            serviceInterface="org.apache.camel.component.jms.remoting.ISay"/>
 
     <!-- export the service -->