- Added task to browse a queue for messages that includes the ability to query for selected messages only, and specify the properties to view. (Still considering adding one for topics too)

- Redesigned the querying mechanism to allow easy insertion and/or extension of specific querying filters
- Redesigned the output mechanism to allow easy inclusion of different output formatters (i.e. output in command shell friendly format, script friendly format, xml format, etc.)
- Currently have just included a not-so-friendly command shell formatter. :) Have plans to modify this and maybe include an xml formatter and/or script friendly formatter.

PS: I plan to move the console package outside activemq-core and include some unit tests with it sometime in the very near future. I know its kinda messy right know. :)

git-svn-id: https://svn.apache.org/repos/asf/incubator/activemq/trunk@371880 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Adrian T. Co 2006-01-24 10:23:34 +00:00
parent 1356d1549c
commit 4b8822f154
39 changed files with 3679 additions and 765 deletions

View File

@ -40,7 +40,7 @@ import java.util.List;
*/
public class Main {
public static final String TASK_DEFAULT_CLASS = "org.apache.activemq.broker.console.DefaultCommand";
public static final String TASK_DEFAULT_CLASS = "org.apache.activemq.broker.console.command.ShellCommand";
private File activeMQHome;
private ClassLoader classLoader;
@ -74,56 +74,52 @@ public class Main {
if (tokens.isEmpty()) {
return;
}
int tokencnt = tokens.size();
String token = (String) tokens.remove(0);
for (int processedcnt = 0; processedcnt < tokencnt; processedcnt++)
{
int count = tokens.size();
int i = 0;
// Parse for all --extdir and --noDefExt options
while (i < count) {
String token = (String)tokens.get(i);
// If token is an extension dir option
if (token.equals("--extdir")) {
// Process token
count--;
tokens.remove(i);
// If no extension directory is specified, or next token is another option
if (!tokens.isEmpty()) {
token = (String) tokens.remove(0);
if (token.startsWith("-"))
{
System.out.println("Extension directory not specified.");
System.out.println("Ignoring extension directory option.");
continue;
}
} else
{
break;
if (i >= count || ((String)tokens.get(i)).startsWith("-")) {
System.out.println("Extension directory not specified.");
System.out.println("Ignoring extension directory option.");
continue;
}
// Process token
processedcnt++;
// Process extension dir token
count--;
File extDir = new File((String)tokens.remove(i));
if(!canUseExtdir()) {
System.out.println("Extension directory feature not available due to the system classpath being able to load: " + TASK_DEFAULT_CLASS);
System.out.println("Ignoring extension directory option.");
} else
{
// Process extension dir token
File extDir = new File(token);
if (!extDir.isDirectory()) {
System.out.println("Extension directory specified is not valid directory: " + extDir);
System.out.println("Ignoring extension directory option.");
continue;
}
addExtensionDirectory(extDir);
continue;
}
if (!extDir.isDirectory()) {
System.out.println("Extension directory specified is not valid directory: " + extDir);
System.out.println("Ignoring extension directory option.");
continue;
}
addExtensionDirectory(extDir);
} else if (token.equals("--noDefExt")) { // If token is --noDefExt option
System.out.println("Bypassing default ext add.");
count--;
tokens.remove(i);
useDefExt = false;
} else
{
break;
} else {
i++;
}
if (!tokens.isEmpty()) token = (String) tokens.remove(0);
}
}
}
public void runTaskClass(List tokens) throws Throwable {
@ -139,8 +135,6 @@ public class Main {
runTask.invoke(task.newInstance(), new Object[] { args, System.in, System.out });
} catch (InvocationTargetException e) {
throw e.getCause();
} catch (Throwable e) {
throw e;
}
}

View File

@ -1,283 +0,0 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console;
import javax.management.MBeanServerConnection;
import javax.management.ObjectName;
import javax.management.MBeanAttributeInfo;
import javax.management.ObjectInstance;
import java.util.Set;
import java.util.List;
import java.util.HashSet;
import java.util.Iterator;
import java.util.ArrayList;
import java.util.Map;
import java.util.HashMap;
import java.util.StringTokenizer;
import java.util.regex.Pattern;
public class AmqJmxSupport {
public static final String DEFAULT_JMX_DOMAIN = "org.apache.activemq";
public static Set getAllBrokers(MBeanServerConnection server) throws Exception {
return queryMBeans(server, new ObjectName(DEFAULT_JMX_DOMAIN + ":Type=Broker,*"));
}
public static Set getBrokers(MBeanServerConnection server, String brokerName) throws Exception {
return queryMBeans(server, "Type=Broker,BrokerName=" + brokerName + ",*");
}
public static Set queryMBeans(MBeanServerConnection server, List queryList) throws Exception {
Set mbeans;
// If there is no query defined get all mbeans
if (queryList==null || queryList.size()==0) {
ObjectName queryName = new ObjectName(DEFAULT_JMX_DOMAIN + ":*");
mbeans = queryMBeans(server, queryName);
// Parse through all the query strings
} else {
mbeans = new HashSet();
for (Iterator i=queryList.iterator(); i.hasNext();) {
String queryStr = (String)i.next();
mbeans.addAll(queryMBeans(server, queryStr));
}
}
return mbeans;
}
public static Set queryMBeans(MBeanServerConnection server, String queryString) throws Exception {
// Transform string to support regex filtering
List regexProp = new ArrayList();
queryString = transformWildcardQueryToObjectName(queryString, regexProp);
ObjectName queryName = new ObjectName(DEFAULT_JMX_DOMAIN + ":" + queryString);
return filterUsingRegEx(queryMBeans(server, queryName), regexProp);
}
public static Set queryMBeans(MBeanServerConnection server, ObjectName objName) throws Exception {
return server.queryMBeans(objName, null);
}
public static Map queryMBeanAttrs(MBeanServerConnection server, ObjectName mbeanObjName, List attrView) throws Exception {
Map attr = new HashMap();
MBeanAttributeInfo[] attrs = server.getMBeanInfo(mbeanObjName).getAttributes();
// If the mbean has no attribute, print a no attribute message
if (attrs.length == 0) {
return null;
}
// If there is no view specified, get all attributes
if (attrView == null || attrView.isEmpty()) {
for (int i=0; i<attrs.length; i++) {
Object attrVal = server.getAttribute(mbeanObjName, attrs[i].getName());
attr.put(attrs[i].getName(), attrVal);
}
return attr;
}
// Get attributes specified by view
for (int i=0; i<attrs.length; i++) {
if (attrView.contains(attrs[i].getName())) {
Object attrVal = server.getAttribute(mbeanObjName, attrs[i].getName());
attr.put(attrs[i].getName(), attrVal);
}
}
return attr;
}
public static String createQueryString(String query, String param) {
return query.replaceAll("%1", param);
}
public static String createQueryString(String query, List params) {
int count = 1;
for (Iterator i=params.iterator();i.hasNext();) {
query.replaceAll("%" + count++, i.next().toString());
}
return query;
}
public static void printBrokerList(Set brokerList) {
Object[] brokerArray = brokerList.toArray();
System.out.println("List of available brokers:");
for (int i=0; i<brokerArray.length; i++) {
String brokerName = ((ObjectInstance)brokerArray[i]).getObjectName().getKeyProperty("BrokerName");
System.out.println(" " + (i+1) + ".) " + brokerName);
}
}
public static void printMBeanProp(ObjectInstance mbean, List propView) {
// Filter properties to print
if (propView != null && !propView.isEmpty()) {
Map mbeanProps = mbean.getObjectName().getKeyPropertyList();
for (Iterator i=propView.iterator(); i.hasNext();) {
Object key = i.next();
Object val = mbeanProps.get(key);
if (val != null) {
System.out.println("MBean " + key + ": " + val);
}
}
// Print all properties
} else {
Map mbeanProps = mbean.getObjectName().getKeyPropertyList();
for (Iterator i=mbeanProps.keySet().iterator(); i.hasNext();) {
Object key = i.next();
Object val = mbeanProps.get(key);
System.out.println("MBean " + key + ": " + val);
}
}
}
public static void printMBeanAttr(MBeanServerConnection server, ObjectInstance mbean, List attrView) {
try {
Map attrList = queryMBeanAttrs(server, mbean.getObjectName(), attrView);
// If the mbean has no attribute, print a no attribute message
if (attrList == null) {
System.out.println(" MBean has no attributes.");
System.out.println();
return;
}
// If the mbean's attributes did not match any of the view, display a message
if (attrList.isEmpty()) {
System.out.println(" View did not match any of the mbean's attributes.");
System.out.println("");
return;
}
// Display mbean attributes
// If attrView is available, use it. This allows control over the display order
if (attrView != null && !attrView.isEmpty()) {
for (Iterator i=attrView.iterator(); i.hasNext();) {
Object key = i.next();
Object val = attrList.get(key);
if (val != null) {
System.out.println(" " + key + " = " + attrList.get(key));
}
}
// If attrView is not available, print all attributes
} else {
for (Iterator i=attrList.keySet().iterator(); i.hasNext();) {
Object key = i.next();
System.out.println(" " + key + " = " + attrList.get(key));
}
}
System.out.println("");
} catch (Exception e) {
System.out.println("Failed to print mbean attributes. Reason: " + e.getMessage());
}
}
private static String transformWildcardQueryToObjectName(String query, List regExMap) throws Exception {
if (regExMap==null) {
regExMap = new ArrayList();
}
StringBuffer newQueryStr = new StringBuffer();
for (StringTokenizer tokenizer = new StringTokenizer(query, ","); tokenizer.hasMoreTokens();) {
String token = tokenizer.nextToken();
// Get key value pair
String key = token;
String value = "";
int pos = key.indexOf("=");
if (pos >= 0) {
value = key.substring(pos + 1);
key = key.substring(0, pos);
}
// Check if value is a wildcard query
if ((value.indexOf("*") >= 0) || (value.indexOf("?") >= 0)) {
// If value is a wildcard query, convert to regex
// and remove the object name query to ensure it selects all
regExMap.add(Pattern.compile("(.*)(" + key + "=)(" + transformWildcardQueryToRegEx(value) + ")(,)(.*)"));
// Re-add valid key value pair. Remove all * property and just add one at the end.
} else if ((key != "") && (value != "")) {
newQueryStr.append(key + "=" + value + ",");
}
}
newQueryStr.append("*");
return newQueryStr.toString();
}
private static String transformWildcardQueryToRegEx(String query) {
query = query.replaceAll("[.]", "\\\\."); // Escape all dot characters. From (.) to (\.)
query = query.replaceAll("[?]", ".");
query = query.replaceAll("[*]", ".*?"); // Use reluctant quantifier
return query;
}
private static Set filterUsingRegEx(Set mbeans, List regexProp) {
// No regular expressions filtering needed
if (regexProp==null || regexProp.isEmpty()) {
return mbeans;
}
Set filteredMbeans = new HashSet();
// Get each bean to filter
for (Iterator i=mbeans.iterator(); i.hasNext();) {
ObjectInstance mbeanInstance = (ObjectInstance)i.next();
String mbeanName = mbeanInstance.getObjectName().getKeyPropertyListString();
// Ensure name ends with ,* to guarantee correct parsing behavior
if (!mbeanName.endsWith(",*")) {
mbeanName = mbeanName + ",*";
}
boolean match = true;
// Match the object name to each regex
for (Iterator j=regexProp.iterator(); j.hasNext();) {
Pattern p = (Pattern)j.next();
if (!p.matcher(mbeanName).matches()) {
match = false;
break;
}
}
// If name of mbean matches all regex pattern, add it
if (match) {
filteredMbeans.add(mbeanInstance);
}
}
return filteredMbeans;
}
}

View File

@ -0,0 +1,64 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console;
import org.apache.activemq.broker.console.filter.QueryFilter;
import org.apache.activemq.broker.console.filter.WildcardToMsgSelectorTransformFilter;
import org.apache.activemq.broker.console.filter.MessagesQueryFilter;
import org.apache.activemq.broker.console.filter.PropertiesViewFilter;
import org.apache.activemq.broker.console.filter.StubQueryFilter;
import org.apache.activemq.broker.console.filter.MapTransformFilter;
import org.apache.activemq.broker.console.filter.GroupPropertiesViewFilter;
import javax.jms.Destination;
import java.net.URI;
import java.util.List;
import java.util.Set;
public class AmqMessagesUtil {
public static final String JMS_MESSAGE_HEADER_PREFIX = "JMS_HEADER_FIELD:";
public static final String JMS_MESSAGE_CUSTOM_PREFIX = "JMS_CUSTOM_FIELD:";
public static final String JMS_MESSAGE_BODY_PREFIX = "JMS_BODY_FIELD:";
public static List getAllMessages(URI brokerUrl, Destination dest) throws Exception {
return getMessages(brokerUrl, dest, "");
}
public static List getMessages(URI brokerUrl, Destination dest, String selector) throws Exception {
return createMessageQueryFilter(brokerUrl, dest).query(selector);
}
public static List getMessages(URI brokerUrl, Destination dest, List selectors) throws Exception {
return createMessageQueryFilter(brokerUrl, dest).query(selectors);
}
public static List filterMessagesView(List messages, Set groupViews, Set attributeViews) throws Exception {
return (new PropertiesViewFilter(attributeViews,
new GroupPropertiesViewFilter(groupViews,
new MapTransformFilter(
new StubQueryFilter(messages)
)
)
)).query("");
}
public static QueryFilter createMessageQueryFilter(URI brokerUrl, Destination dest) {
return new WildcardToMsgSelectorTransformFilter(
new MessagesQueryFilter(brokerUrl, dest)
);
}
}

View File

@ -1,66 +0,0 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console;
import java.util.List;
public class DefaultCommand extends AbstractCommand {
protected void execute(List tokens) {
// Process task token
if( tokens.size() > 0 ) {
String taskToken = (String)tokens.remove(0);
if (taskToken.equals("start")) {
new StartCommand().execute(tokens);
} else if (taskToken.equals("stop")) {
new ShutdownCommand().execute(tokens);
} else if (taskToken.equals("list")) {
new ListCommand().execute(tokens);
} else if (taskToken.equals("query")) {
new QueryCommand().execute(tokens);
} else {
// If not valid task, push back to list
tokens.add(0, taskToken);
new StartCommand().execute(tokens);
}
} else {
new StartCommand().execute(tokens);
}
}
protected void printHelp() {
out.println("Usage: Main [task] [--extdir <dir>] [task-options] [task data]");
out.println("");
out.println("Tasks (default task is start):");
out.println(" start - Creates and starts a broker using a configuration file, or a broker URI.");
out.println(" stop - Stops a running broker specified by the broker name.");
out.println(" list - Lists all available brokers in the specified JMX context.");
out.println(" query - Display selected broker component's attributes and statistics.");
out.println(" --extdir <dir> - Add the jar files in the directory to the classpath.");
out.println(" --version - Display the version information.");
out.println(" -h,-?,--help - Display this help information. To display task specific help, use Main [task] -h,-?,--help");
out.println("");
out.println("Task Options:");
out.println(" - Properties specific to each task.");
out.println("");
out.println("Task Data:");
out.println(" - Information needed by each specific task.");
out.println("");
}
}

View File

@ -0,0 +1,116 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console;
import org.apache.activemq.broker.console.filter.QueryFilter;
import org.apache.activemq.broker.console.filter.MBeansObjectNameQueryFilter;
import org.apache.activemq.broker.console.filter.WildcardToRegExTransformFilter;
import org.apache.activemq.broker.console.filter.MBeansRegExQueryFilter;
import org.apache.activemq.broker.console.filter.MBeansAttributeQueryFilter;
import org.apache.activemq.broker.console.filter.PropertiesViewFilter;
import org.apache.activemq.broker.console.filter.StubQueryFilter;
import org.apache.activemq.broker.console.filter.MapTransformFilter;
import javax.management.remote.JMXServiceURL;
import java.util.Set;
import java.util.List;
import java.util.Iterator;
public class JmxMBeansUtil {
public static List getAllBrokers(JMXServiceURL jmxUrl) throws Exception {
return (new MBeansObjectNameQueryFilter(jmxUrl)).query("Type=Broker");
}
public static List getBrokersByName(JMXServiceURL jmxUrl, String brokerName) throws Exception {
return (new MBeansObjectNameQueryFilter(jmxUrl)).query("Type=Broker,BrokerName=" + brokerName);
}
public static List getAllBrokers(JMXServiceURL jmxUrl, Set attributes) throws Exception {
return (new MBeansAttributeQueryFilter(jmxUrl, attributes, new MBeansObjectNameQueryFilter(jmxUrl))).query("Type=Broker");
}
public static List getBrokersByName(JMXServiceURL jmxUrl, String brokerName, Set attributes) throws Exception {
return (new MBeansAttributeQueryFilter(jmxUrl, attributes, new MBeansObjectNameQueryFilter(jmxUrl))).query("Type=Broker,BrokerName=" + brokerName);
}
public static List queryMBeans(JMXServiceURL jmxUrl, List queryList) throws Exception {
// If there is no query defined get all mbeans
if (queryList==null || queryList.size()==0) {
return createMBeansObjectNameQuery(jmxUrl).query("");
// Parse through all the query strings
} else {
return createMBeansObjectNameQuery(jmxUrl).query(queryList);
}
}
public static List queryMBeans(JMXServiceURL jmxUrl, List queryList, Set attributes) throws Exception {
// If there is no query defined get all mbeans
if (queryList==null || queryList.size()==0) {
return createMBeansAttributeQuery(jmxUrl, attributes).query("");
// Parse through all the query strings
} else {
return createMBeansAttributeQuery(jmxUrl, attributes).query(queryList);
}
}
public static List queryMBeans(JMXServiceURL jmxUrl, String queryString) throws Exception {
return createMBeansObjectNameQuery(jmxUrl).query(queryString);
}
public static List queryMBeans(JMXServiceURL jmxUrl, String queryString, Set attributes) throws Exception {
return createMBeansAttributeQuery(jmxUrl, attributes).query(queryString);
}
public static List filterMBeansView(List mbeans, Set viewFilter) throws Exception {
return (new PropertiesViewFilter(viewFilter, new MapTransformFilter(new StubQueryFilter(mbeans))).query(""));
}
public static String createQueryString(String query, String param) {
return query.replaceAll("%1", param);
}
public static String createQueryString(String query, List params) {
int count = 1;
for (Iterator i=params.iterator();i.hasNext();) {
query.replaceAll("%" + count++, i.next().toString());
}
return query;
}
public static QueryFilter createMBeansObjectNameQuery(JMXServiceURL jmxUrl) {
return new WildcardToRegExTransformFilter( // Let us be able to accept wildcard queries
new MBeansRegExQueryFilter( // Use regular expressions to filter the query results
new MBeansObjectNameQueryFilter(jmxUrl) // Let us retrieve the mbeans object name specified by the query
)
);
}
public static QueryFilter createMBeansAttributeQuery(JMXServiceURL jmxUrl, Set attributes) {
return new WildcardToRegExTransformFilter( // Let use be able to accept wildcard queries
new MBeansRegExQueryFilter( // Use regular expressions to filter the query result
new MBeansAttributeQueryFilter(jmxUrl, attributes, // Retrieve the attributes needed
new MBeansObjectNameQueryFilter(jmxUrl) // Retrieve the mbeans object name specified by the query
)
)
);
}
}

View File

@ -1,43 +0,0 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console;
import java.util.List;
public class ListCommand extends AbstractJmxCommand {
protected void execute(List tokens) {
try {
AmqJmxSupport.printBrokerList(AmqJmxSupport.getAllBrokers(createJmxConnector().getMBeanServerConnection()));
closeJmxConnector();
} catch (Throwable e) {
System.out.println("Failed to execute list task. Reason: " + e);
}
}
protected void printHelp() {
System.out.println("Task Usage: Main list [list-options]");
System.out.println("Description: Lists all available broker in the specified JMX context.");
System.out.println("");
System.out.println("List Options:");
System.out.println(" --jmxurl <url> Set the JMX URL to connect to.");
System.out.println(" --version Display the version information.");
System.out.println(" -h,-?,--help Display the stop broker help information.");
System.out.println("");
}
}

View File

@ -1,206 +0,0 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console;
import javax.management.remote.JMXConnector;
import javax.management.MBeanServerConnection;
import javax.management.ObjectInstance;
import java.util.List;
import java.util.Properties;
import java.util.ArrayList;
import java.util.Enumeration;
import java.util.StringTokenizer;
import java.util.Set;
import java.util.Iterator;
public class QueryCommand extends AbstractJmxCommand {
// Predefined type=identifier query
private static final Properties PREDEFINED_OBJNAME_QUERY = new Properties();
static {
PREDEFINED_OBJNAME_QUERY.setProperty("Broker", "Type=Broker,BrokerName=%1,*");
PREDEFINED_OBJNAME_QUERY.setProperty("Connection", "Type=Connection,Connection=%1,*");
PREDEFINED_OBJNAME_QUERY.setProperty("Connector", "Type=Connector,ConnectorName=%1,*");
PREDEFINED_OBJNAME_QUERY.setProperty("NetworkConnector", "Type=NetworkConnector,BrokerName=%1,*");
PREDEFINED_OBJNAME_QUERY.setProperty("Queue", "Type=Queue,Destination=%1,*");
PREDEFINED_OBJNAME_QUERY.setProperty("Topic", "Type=Topic,Destination=%1,*");
};
private final List queryAddObjects = new ArrayList(10);
private final List querySubObjects = new ArrayList(10);
private final List queryViews = new ArrayList(10);
protected void execute(List tokens) {
try {
// Connect to jmx server
JMXConnector jmxConnector = createJmxConnector();
MBeanServerConnection server = jmxConnector.getMBeanServerConnection();
// Query for the mbeans to add
Set addMBeans = AmqJmxSupport.queryMBeans(server, queryAddObjects);
// Query for the mbeans to sub
if (querySubObjects.size() > 0) {
Set subMBeans = AmqJmxSupport.queryMBeans(server, querySubObjects);
addMBeans.removeAll(subMBeans);
}
for (Iterator i=addMBeans.iterator(); i.hasNext();) {
ObjectInstance mbean = (ObjectInstance)i.next();
AmqJmxSupport.printMBeanProp(mbean, null);
AmqJmxSupport.printMBeanAttr(server, mbean, queryViews);
}
closeJmxConnector();
} catch (Throwable e) {
System.out.println("Failed to execute query task. Reason: " + e);
}
}
protected void handleOption(String token, List tokens) throws Exception {
// If token is a additive predefined query define option
if (token.startsWith("-Q")) {
String key = token.substring(2);
String value = "";
int pos = key.indexOf("=");
if (pos >= 0) {
value = key.substring(pos + 1);
key = key.substring(0, pos);
}
// If additive query
String predefQuery = PREDEFINED_OBJNAME_QUERY.getProperty(key);
if (predefQuery == null) {
printError("Unknown query object type: " + key);
return;
}
String queryStr = AmqJmxSupport.createQueryString(predefQuery, value);
queryAddObjects.add(queryStr);
}
// If token is a substractive predefined query define option
else if (token.startsWith("-xQ")) {
String key = token.substring(3);
String value = "";
int pos = key.indexOf("=");
if (pos >= 0) {
value = key.substring(pos + 1);
key = key.substring(0, pos);
}
// If subtractive query
String predefQuery = PREDEFINED_OBJNAME_QUERY.getProperty(key);
if (predefQuery == null) {
printError("Unknown query object type: " + key);
return;
}
String queryStr = AmqJmxSupport.createQueryString(predefQuery, value);
querySubObjects.add(queryStr);
}
// If token is an additive object name query option
else if (token.startsWith("--objname")) {
// If no object name query is specified, or next token is a new option
if (tokens.isEmpty() || ((String)tokens.get(0)).startsWith("-")) {
printError("Object name query not specified");
return;
}
String queryString = (String)tokens.remove(0);
queryAddObjects.add(queryString);
}
// If token is a substractive object name query option
else if (token.startsWith("--xobjname")) {
// If no object name query is specified, or next token is a new option
if (tokens.isEmpty() || ((String)tokens.get(0)).startsWith("-")) {
printError("Object name query not specified");
return;
}
String queryString = (String)tokens.remove(0);
querySubObjects.add(queryString);
}
// If token is a view option
else if (token.startsWith("--view")) {
// If no view specified, or next token is a new option
if (tokens.isEmpty() || ((String)tokens.get(0)).startsWith("-")) {
printError("Attributes to view not specified");
return;
}
// Add the attributes to view
Enumeration viewTokens = new StringTokenizer((String)tokens.remove(0), ",", false);
while (viewTokens.hasMoreElements()) {
queryViews.add(viewTokens.nextElement());
}
}
// Let super class handle unknown option
else {
super.handleOption(token, tokens);
}
}
protected void printHelp() {
System.out.println("Task Usage: Main query [query-options]");
System.out.println("Description: Display selected broker component's attributes and statistics.");
System.out.println("");
System.out.println("Query Options:");
System.out.println(" -Q<type>=<name> Add to the search list the specific object type matched by the defined object identifier.");
System.out.println(" -xQ<type>=<name> Remove from the search list the specific object type matched by the object identifier.");
System.out.println(" --objname <query> Add to the search list objects matched by the query similar to the JMX object name format.");
System.out.println(" --xobjname <query> Remove from the search list objects matched by the query similar to the JMX object name format.");
System.out.println(" --view <attr1>,<attr2>,... Select the specific attribute of the object to view. By default all attributes will be displayed.");
System.out.println(" --jmxurl <url> Set the JMX URL to connect to.");
System.out.println(" --version Display the version information.");
System.out.println(" -h,-?,--help Display the query broker help information.");
System.out.println("");
System.out.println("Examples:");
System.out.println(" Main query");
System.out.println(" - Print all the attributes of all registered objects (queues, topics, connections, etc).");
System.out.println("");
System.out.println(" Main query -QQueue=TEST.FOO");
System.out.println(" - Print all the attributes of the queue with destination name TEST.FOO.");
System.out.println("");
System.out.println(" Main query -QTopic=*");
System.out.println(" - Print all the attributes of all registered topics.");
System.out.println("");
System.out.println(" Main query --view EnqueueCount,DequeueCount");
System.out.println(" - Print the attributes EnqueueCount and DequeueCount of all registered objects.");
System.out.println("");
System.out.println(" Main -QTopic=* --view EnqueueCount,DequeueCount");
System.out.println(" - Print the attributes EnqueueCount and DequeueCount of all registered topics.");
System.out.println("");
System.out.println(" Main -QTopic=* -QQueue=* --view EnqueueCount,DequeueCount");
System.out.println(" - Print the attributes EnqueueCount and DequeueCount of all registered topics and queues.");
System.out.println("");
System.out.println(" Main -QTopic=* -xQTopic=ActiveMQ.Advisory.*");
System.out.println(" - Print all attributes of all topics except those that has a name that begins with \"ActiveMQ.Advisory\".");
System.out.println("");
System.out.println(" Main --objname Type=*Connect*,BrokerName=local* -xQNetworkConnector=*");
System.out.println(" - Print all attributes of all connectors, connections excluding network connectors that belongs to the broker that begins with local.");
System.out.println("");
System.out.println(" Main -QQueue=* -xQQueue=????");
System.out.println(" - Print all attributes of all queues except those that are 4 letters long.");
System.out.println("");
}
}

View File

@ -0,0 +1,156 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.command;
import org.apache.activemq.ActiveMQConnectionFactory;
import org.apache.activemq.broker.console.formatter.GlobalWriter;
import javax.jms.ConnectionFactory;
import javax.jms.Connection;
import javax.jms.JMSException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.List;
import java.util.ArrayList;
import java.util.Iterator;
public abstract class AbstractAmqCommand extends AbstractCommand {
private URI brokerUrl;
private ConnectionFactory factory;
private final List connections = new ArrayList();
/**
* Establishes a connection to the remote broker specified by the broker url.
* @return - connection to the broker
* @throws JMSException
*/
protected Connection createConnection() throws JMSException {
if (getBrokerUrl() == null) {
GlobalWriter.printException(new IllegalStateException("You must specify a broker URL to connect to using the --amqurl option."));
return null;
}
if (factory == null) {
factory = new ActiveMQConnectionFactory(getBrokerUrl());
}
Connection conn = factory.createConnection();
connections.add(conn);
return conn;
}
/**
* Establishes a connection to the remote broker specified by the broker url.
* @param username - username for the connection
* @param password - password for the connection
* @return - connection to the broker
* @throws JMSException
*/
protected Connection createConnection(String username, String password) throws JMSException {
if (getBrokerUrl() == null) {
GlobalWriter.printException(new IllegalStateException("You must specify a broker URL to connect to using the --amqurl option."));
return null;
}
if (factory == null) {
factory = new ActiveMQConnectionFactory(getBrokerUrl());
}
Connection conn = factory.createConnection(username, password);
connections.add(conn);
conn.start();
return conn;
}
/**
* Close all created connections.
*/
protected void closeAllConnections() {
for (Iterator i=connections.iterator(); i.hasNext();) {
try {
((Connection)i.next()).close();
} catch (Exception e) { }
}
connections.clear();
}
/**
* Handle the --amqurl option.
* @param token - current option
* @param tokens - succeeding list of arguments
* @throws Exception
*/
protected void handleOption(String token, List tokens) throws Exception {
// Try to handle the options first
if (token.equals("--amqurl")) {
// If no broker url specified, or next token is a new option
if (tokens.isEmpty() || ((String)tokens.get(0)).startsWith("-")) {
GlobalWriter.printException(new IllegalArgumentException("Broker URL not specified."));
tokens.clear();
return;
}
// If broker url already specified
if (getBrokerUrl() != null) {
GlobalWriter.printException(new IllegalArgumentException("Multiple broker URL cannot be specified."));
tokens.clear();
return;
}
String strBrokerUrl = (String)tokens.remove(0);
try {
setBrokerUrl(new URI(strBrokerUrl));
} catch (URISyntaxException e) {
GlobalWriter.printException(e);
tokens.clear();
return;
}
} else {
// Let the super class handle the option
super.handleOption(token, tokens);
}
}
/**
* Set the broker url.
* @param brokerUrl - new broker url
*/
protected void setBrokerUrl(URI brokerUrl) {
this.brokerUrl = brokerUrl;
}
/**
* Set the broker url.
* @param address - address of the new broker url
* @throws URISyntaxException
*/
protected void setBrokerUrl(String address) throws URISyntaxException {
this.brokerUrl = new URI(address);
}
/**
* Get the current broker url.
* @return current broker url
*/
protected URI getBrokerUrl() {
return brokerUrl;
}
}

View File

@ -14,42 +14,47 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.activemq.broker.console;
package org.apache.activemq.broker.console.command;
import org.apache.activemq.ActiveMQConnectionMetaData;
import org.apache.activemq.broker.console.formatter.GlobalWriter;
import java.io.InputStream;
import java.io.PrintStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
public abstract class AbstractCommand implements Command {
public static final String COMMAND_OPTION_DELIMETER = ",";
private boolean isPrintHelp = false;
private boolean isPrintVersion = false;
protected PrintStream out;
public int main(String[] args, InputStream in, PrintStream out) {
this.out = out;
try {
List tokens = new ArrayList(Arrays.asList(args));
parseOptions(tokens);
/**
* Exceute a generic command, which includes parsing the options for the command and running the specific task.
* @param tokens - command arguments
* @throws Exception
*/
public void execute(List tokens) throws Exception {
// Parse the options specified by "-"
parseOptions(tokens);
if (isPrintHelp) {
printHelp();
} else if (isPrintVersion) {
printVersion();
} else {
execute(tokens);
}
return 0;
} catch (Exception e) {
out.println("Failed to execute main task. Reason: " + e);
e.printStackTrace(out);
return -1;
// Print the help file of the task
if (isPrintHelp) {
printHelp();
// Print the AMQ version
} else if (isPrintVersion) {
GlobalWriter.printVersion(ActiveMQConnectionMetaData.PROVIDER_VERSION);
// Run the specified task
} else {
runTask(tokens);
}
}
/**
* Parse any option parameters in the command arguments specified by a '-' as the first character of the token.
* @param tokens - command arguments
* @throws Exception
*/
protected void parseOptions(List tokens) throws Exception {
while (!tokens.isEmpty()) {
String token = (String)tokens.remove(0);
@ -64,6 +69,12 @@ public abstract class AbstractCommand implements Command {
}
}
/**
* Handle the general options for each command, which includes -h, -?, --help, -D, --version.
* @param token - option token to handle
* @param tokens - succeeding command arguments
* @throws Exception
*/
protected void handleOption(String token, List tokens) throws Exception {
// If token is a help option
if (token.equals("-h") || token.equals("-?") || token.equals("--help")) {
@ -91,23 +102,19 @@ public abstract class AbstractCommand implements Command {
// Token is unrecognized
else {
out.println("Ignoring unrecognized option: " + token);
GlobalWriter.printInfo("Ignoring unrecognized option: " + token);
}
}
protected void printVersion() {
out.println();
out.println("ActiveMQ " + ActiveMQConnectionMetaData.PROVIDER_VERSION);
out.println("For help or more information please see: http://www.logicblaze.com");
out.println();
}
/**
* Run the specific task.
* @param tokens - command arguments
* @throws Exception
*/
abstract protected void runTask(List tokens) throws Exception;
protected void printError(String message) {
isPrintHelp = true;
out.println(message);
out.println();
}
abstract protected void execute(List tokens) throws Exception;
/**
* Print the help messages for the specific task
*/
abstract protected void printHelp();
}

View File

@ -14,7 +14,9 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.activemq.broker.console;
package org.apache.activemq.broker.console.command;
import org.apache.activemq.broker.console.formatter.GlobalWriter;
import javax.management.remote.JMXServiceURL;
import javax.management.remote.JMXConnector;
@ -29,19 +31,51 @@ public abstract class AbstractJmxCommand extends AbstractCommand {
private JMXServiceURL jmxServiceUrl;
private JMXConnector jmxConnector;
protected JMXServiceURL getJmxServiceUrl() throws Exception {
/**
* Get the current specified JMX service url.
* @return JMX service url
*/
protected JMXServiceURL getJmxServiceUrl() {
return jmxServiceUrl;
}
/**
* Get the current JMX service url being used, or create a default one if no JMX service url has been specified.
* @return JMX service url
* @throws MalformedURLException
*/
protected JMXServiceURL useJmxServiceUrl() throws MalformedURLException {
if (getJmxServiceUrl() == null) {
setJmxServiceUrl(DEFAULT_JMX_URL);
}
return getJmxServiceUrl();
}
/**
* Sets the JMX service url to use.
* @param jmxServiceUrl - new JMX service url to use
*/
protected void setJmxServiceUrl(JMXServiceURL jmxServiceUrl) {
this.jmxServiceUrl = jmxServiceUrl;
}
protected void setJmxServiceUrl(String jmxServiceUrl) throws Exception {
/**
* Sets the JMX service url to use.
* @param jmxServiceUrl - new JMX service url to use
* @throws MalformedURLException
*/
protected void setJmxServiceUrl(String jmxServiceUrl) throws MalformedURLException {
setJmxServiceUrl(new JMXServiceURL(jmxServiceUrl));
}
protected JMXConnector createJmxConnector() throws Exception {
/**
* Create a JMX connector using the current specified JMX service url. If there is an existing connection,
* it tries to reuse this connection.
* @return created JMX connector
* @throws IOException
*/
protected JMXConnector createJmxConnector() throws IOException {
// Reuse the previous connection
if (jmxConnector != null) {
jmxConnector.connect();
@ -49,13 +83,13 @@ public abstract class AbstractJmxCommand extends AbstractCommand {
}
// Create a new JMX connector
if (getJmxServiceUrl() == null) {
setJmxServiceUrl(DEFAULT_JMX_URL);
}
jmxConnector = JMXConnectorFactory.connect(getJmxServiceUrl());
jmxConnector = JMXConnectorFactory.connect(useJmxServiceUrl());
return jmxConnector;
}
/**
* Close the current JMX connector
*/
protected void closeJmxConnector() {
try {
if (jmxConnector != null) {
@ -66,17 +100,23 @@ public abstract class AbstractJmxCommand extends AbstractCommand {
}
}
/**
* Handle the --jmxurl option.
* @param token - option token to handle
* @param tokens - succeeding command arguments
* @throws Exception
*/
protected void handleOption(String token, List tokens) throws Exception {
// Try to handle the options first
if (token.equals("--jmxurl")) {
// If no jmx url specified, or next token is a new option
if (tokens.isEmpty() || ((String)tokens.get(0)).startsWith("-")) {
printError("JMX URL not specified.");
GlobalWriter.printException(new IllegalArgumentException("JMX URL not specified."));
}
// If jmx url already specified
if (getJmxServiceUrl() != null) {
printError("Multiple JMX URL cannot be specified.");
GlobalWriter.printException(new IllegalArgumentException("Multiple JMX URL cannot be specified."));
tokens.clear();
}
@ -84,7 +124,7 @@ public abstract class AbstractJmxCommand extends AbstractCommand {
try {
this.setJmxServiceUrl(new JMXServiceURL(strJmxUrl));
} catch (MalformedURLException e) {
printError("Invalid JMX URL format: " + strJmxUrl);
GlobalWriter.printException(e);
tokens.clear();
}
} else {

View File

@ -0,0 +1,240 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.command;
import org.apache.activemq.broker.console.AmqMessagesUtil;
import org.apache.activemq.broker.console.formatter.GlobalWriter;
import org.apache.activemq.command.ActiveMQQueue;
import org.apache.activemq.command.ActiveMQTopic;
import javax.jms.Destination;
import java.util.List;
import java.util.ArrayList;
import java.util.Set;
import java.util.HashSet;
import java.util.StringTokenizer;
import java.util.Iterator;
public class BrowseCommand extends AbstractAmqCommand {
public static final String QUEUE_PREFIX = "queue:";
public static final String TOPIC_PREFIX = "topic:";
public static final String VIEW_GROUP_HEADER = "header:";
public static final String VIEW_GROUP_CUSTOM = "custom:";
public static final String VIEW_GROUP_BODY = "body:";
private final List queryAddObjects = new ArrayList(10);
private final List querySubObjects = new ArrayList(10);
private final Set groupViews = new HashSet(10);
private final Set queryViews = new HashSet(10);
/**
* Execute the browse command, which allows you to browse the messages in a given JMS destination
* @param tokens - command arguments
* @throws Exception
*/
protected void runTask(List tokens) throws Exception {
try {
// If no destination specified
if (tokens.isEmpty()) {
GlobalWriter.printException(new IllegalArgumentException("No JMS destination specified."));
return;
}
// If no broker url specified
if (getBrokerUrl() == null) {
GlobalWriter.printException(new IllegalStateException("No broker url specified. Use the --amqurl option to specify a broker url."));
return;
}
// Display the messages for each destination
for (Iterator i=tokens.iterator(); i.hasNext();) {
String destName = (String)i.next();
Destination dest;
// If destination has been explicitly specified as a queue
if (destName.startsWith(QUEUE_PREFIX)) {
dest = new ActiveMQQueue(destName.substring(QUEUE_PREFIX.length()));
// If destination has been explicitly specified as a topic
} else if (destName.startsWith(TOPIC_PREFIX)) {
dest = new ActiveMQTopic(destName.substring(TOPIC_PREFIX.length()));
// By default destination is assumed to be a queue
} else {
dest = new ActiveMQQueue(destName);
}
// Query for the messages to view
List addMsgs = AmqMessagesUtil.getMessages(getBrokerUrl(), dest, queryAddObjects);
// Query for the messages to remove from view
if (querySubObjects.size() > 0) {
List subMsgs = AmqMessagesUtil.getMessages(getBrokerUrl(), dest, querySubObjects);
addMsgs.removeAll(subMsgs);
}
// Display the messages
GlobalWriter.printMessage(AmqMessagesUtil.filterMessagesView(addMsgs, groupViews, queryViews));
}
} catch (Exception e) {
GlobalWriter.printException(new RuntimeException("Failed to execute browse task. Reason: " + e));
throw new Exception(e);
}
}
/**
* Handle the --msgsel, --xmsgsel, --view, -V options.
* @param token - option token to handle
* @param tokens - succeeding command arguments
* @throws Exception
*/
protected void handleOption(String token, List tokens) throws Exception {
// If token is an additive message selector option
if (token.startsWith("--msgsel")) {
// If no message selector is specified, or next token is a new option
if (tokens.isEmpty() || ((String)tokens.get(0)).startsWith("-")) {
GlobalWriter.printException(new IllegalArgumentException("Message selector not specified"));
return;
}
StringTokenizer queryTokens = new StringTokenizer((String)tokens.remove(0), COMMAND_OPTION_DELIMETER);
while (queryTokens.hasMoreTokens()) {
queryAddObjects.add(queryTokens.nextToken());
}
}
// If token is a substractive message selector option
else if (token.startsWith("--xmsgsel")) {
// If no message selector is specified, or next token is a new option
if (tokens.isEmpty() || ((String)tokens.get(0)).startsWith("-")) {
GlobalWriter.printException(new IllegalArgumentException("Message selector not specified"));
return;
}
StringTokenizer queryTokens = new StringTokenizer((String)tokens.remove(0), COMMAND_OPTION_DELIMETER);
while (queryTokens.hasMoreTokens()) {
querySubObjects.add(queryTokens.nextToken());
}
}
// If token is a view option
else if (token.startsWith("--view")) {
// If no view specified, or next token is a new option
if (tokens.isEmpty() || ((String)tokens.get(0)).startsWith("-")) {
GlobalWriter.printException(new IllegalArgumentException("Attributes to view not specified"));
return;
}
// Add the attributes to view
StringTokenizer viewTokens = new StringTokenizer((String)tokens.remove(0), COMMAND_OPTION_DELIMETER);
while (viewTokens.hasMoreTokens()) {
String viewToken = viewTokens.nextToken();
// If view is explicitly specified to belong to the JMS header
if (viewToken.equals(VIEW_GROUP_HEADER)) {
queryViews.add(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX + viewToken.substring(VIEW_GROUP_HEADER.length()));
// If view is explicitly specified to belong to the JMS custom header
} else if (viewToken.equals(VIEW_GROUP_CUSTOM)) {
queryViews.add(AmqMessagesUtil.JMS_MESSAGE_CUSTOM_PREFIX + viewToken.substring(VIEW_GROUP_CUSTOM.length()));
// If view is explicitly specified to belong to the JMS body
} else if (viewToken.equals(VIEW_GROUP_BODY)) {
queryViews.add(AmqMessagesUtil.JMS_MESSAGE_BODY_PREFIX + viewToken.substring(VIEW_GROUP_BODY.length()));
// If no view explicitly specified, let's check the view for each group
} else {
queryViews.add(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX + viewToken);
queryViews.add(AmqMessagesUtil.JMS_MESSAGE_CUSTOM_PREFIX + viewToken);
queryViews.add(AmqMessagesUtil.JMS_MESSAGE_BODY_PREFIX + viewToken);
}
}
}
// If token is a predefined group view option
else if (token.startsWith("-V")) {
String viewGroup = token.substring(2);
// If option is a header group view
if (viewGroup.equals("header")) {
groupViews.add(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX);
// If option is a custom header group view
} else if (viewGroup.equals("custom")) {
groupViews.add(AmqMessagesUtil.JMS_MESSAGE_CUSTOM_PREFIX);
// If option is a body group view
} else if (viewGroup.equals("body")) {
groupViews.add(AmqMessagesUtil.JMS_MESSAGE_BODY_PREFIX);
// Unknown group view
} else {
GlobalWriter.printInfo("Unknown group view: " + viewGroup + ". Ignoring group view option.");
}
}
// Let super class handle unknown option
else {
super.handleOption(token, tokens);
}
}
/**
* Print the help messages for the browse command
*/
protected void printHelp() {
GlobalWriter.printHelp(helpFile);
}
protected String[] helpFile = new String[] {
"Task Usage: Main browse --amqurl <broker url> [browse-options] <destinations>",
"Description: Display selected destination's messages.",
"",
"Browse Options:",
" --amqurl <url> Set the broker URL to connect to.",
" --msgsel <msgsel1,msglsel2> Add to the search list messages matched by the query similar to",
" the messages selector format.",
" -V<header|custom|body> Predefined view that allows you to view the message header, custom",
" message header, or the message body.",
" --view <attr1>,<attr2>,... Select the specific attribute of the message to view.",
" --version Display the version information.",
" -h,-?,--help Display the browse broker help information.",
"",
"Examples:",
" Main browse --amqurl tcp://localhost:61616 FOO.BAR",
" - Print the message header, custom message header, and message body of all messages in the",
" queue FOO.BAR",
"",
" Main browse --amqurl tcp://localhost:61616 -Vheader,body queue:FOO.BAR",
" - Print only the message header and message body of all messages in the queue FOO.BAR",
"",
" Main browse --amqurl tcp://localhost:61616 -Vheader --view custom:MyField queue:FOO.BAR",
" - Print the message header and the custom field 'MyField' of all messages in the queue FOO.BAR",
"",
" Main browse --amqurl tcp://localhost:61616 --msgsel JMSMessageID='*:10',JMSPriority>5 FOO.BAR",
" - Print all the message fields that has a JMSMessageID in the header field that matches the",
" wildcard *:10, and has a JMSPriority field > 5 in the queue FOO.BAR",
" * To use wildcard queries, the field must be a string and the query enclosed in ''",
"",
};
}

View File

@ -15,11 +15,15 @@
* limitations under the License.
*/
package org.apache.activemq.broker.console;
package org.apache.activemq.broker.console.command;
import java.io.InputStream;
import java.io.PrintStream;
import java.util.List;
public interface Command {
public int main(String[] args, InputStream in, PrintStream out);
/**
* Execute the specified command
* @param tokens - arguments to the command
* @throws Exception
*/
public void execute(List tokens) throws Exception;
}

View File

@ -0,0 +1,61 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.command;
import org.apache.activemq.broker.console.formatter.GlobalWriter;
import org.apache.activemq.broker.console.JmxMBeansUtil;
import java.util.List;
import java.util.Set;
import java.util.HashSet;
public class ListCommand extends AbstractJmxCommand {
/**
* List all running brokers registered in the specified JMX context
* @param tokens - command arguments
* @throws Exception
*/
protected void runTask(List tokens) throws Exception {
try {
Set propsView = new HashSet();
propsView.add("BrokerName");
GlobalWriter.printMBean(JmxMBeansUtil.filterMBeansView(JmxMBeansUtil.getAllBrokers(useJmxServiceUrl()), propsView));
} catch (Exception e) {
GlobalWriter.printException(new RuntimeException("Failed to execute list task. Reason: " + e));
throw new Exception(e);
}
}
/**
* Print the help messages for the browse command
*/
protected void printHelp() {
GlobalWriter.printHelp(helpFile);
}
protected String[] helpFile = new String[] {
"Task Usage: Main list [list-options]",
"Description: Lists all available broker in the specified JMX context.",
"",
"List Options:",
" --jmxurl <url> Set the JMX URL to connect to.",
" --version Display the version information.",
" -h,-?,--help Display the stop broker help information.",
""
};
}

View File

@ -0,0 +1,229 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.command;
import org.apache.activemq.broker.console.JmxMBeansUtil;
import org.apache.activemq.broker.console.formatter.GlobalWriter;
import java.util.List;
import java.util.Properties;
import java.util.ArrayList;
import java.util.Enumeration;
import java.util.StringTokenizer;
import java.util.Set;
import java.util.HashSet;
public class QueryCommand extends AbstractJmxCommand {
// Predefined type=identifier query
private static final Properties PREDEFINED_OBJNAME_QUERY = new Properties();
static {
PREDEFINED_OBJNAME_QUERY.setProperty("Broker", "Type=Broker,BrokerName=%1,*");
PREDEFINED_OBJNAME_QUERY.setProperty("Connection", "Type=Connection,Connection=%1,*");
PREDEFINED_OBJNAME_QUERY.setProperty("Connector", "Type=Connector,ConnectorName=%1,*");
PREDEFINED_OBJNAME_QUERY.setProperty("NetworkConnector", "Type=NetworkConnector,BrokerName=%1,*");
PREDEFINED_OBJNAME_QUERY.setProperty("Queue", "Type=Queue,Destination=%1,*");
PREDEFINED_OBJNAME_QUERY.setProperty("Topic", "Type=Topic,Destination=%1,*");
};
private final List queryAddObjects = new ArrayList(10);
private final List querySubObjects = new ArrayList(10);
private final Set queryViews = new HashSet(10);
/**
* Queries the mbeans registered in the specified JMX context
* @param tokens - command arguments
* @throws Exception
*/
protected void runTask(List tokens) throws Exception {
try {
// Query for the mbeans to add
List addMBeans = JmxMBeansUtil.queryMBeans(useJmxServiceUrl(), queryAddObjects, queryViews);
// Query for the mbeans to sub
if (querySubObjects.size() > 0) {
List subMBeans = JmxMBeansUtil.queryMBeans(useJmxServiceUrl(), querySubObjects, queryViews);
addMBeans.removeAll(subMBeans);
}
GlobalWriter.printMBean(JmxMBeansUtil.filterMBeansView(addMBeans, queryViews));
} catch (Exception e) {
GlobalWriter.printException(new RuntimeException("Failed to execute query task. Reason: " + e));
throw new Exception(e);
}
}
/**
* Handle the -Q, -xQ, --objname, --xobjname, --view options.
* @param token - option token to handle
* @param tokens - succeeding command arguments
* @throws Exception
*/
protected void handleOption(String token, List tokens) throws Exception {
// If token is a additive predefined query define option
if (token.startsWith("-Q")) {
String key = token.substring(2);
String value = "";
int pos = key.indexOf("=");
if (pos >= 0) {
value = key.substring(pos + 1);
key = key.substring(0, pos);
}
// If additive query
String predefQuery = PREDEFINED_OBJNAME_QUERY.getProperty(key);
if (predefQuery == null) {
GlobalWriter.printException(new IllegalArgumentException("Unknown query object type: " + key));
return;
}
String queryStr = JmxMBeansUtil.createQueryString(predefQuery, value);
queryAddObjects.add(queryStr);
}
// If token is a substractive predefined query define option
else if (token.startsWith("-xQ")) {
String key = token.substring(3);
String value = "";
int pos = key.indexOf("=");
if (pos >= 0) {
value = key.substring(pos + 1);
key = key.substring(0, pos);
}
// If subtractive query
String predefQuery = PREDEFINED_OBJNAME_QUERY.getProperty(key);
if (predefQuery == null) {
GlobalWriter.printException(new IllegalArgumentException("Unknown query object type: " + key));
return;
}
String queryStr = JmxMBeansUtil.createQueryString(predefQuery, value);
querySubObjects.add(queryStr);
}
// If token is an additive object name query option
else if (token.startsWith("--objname")) {
// If no object name query is specified, or next token is a new option
if (tokens.isEmpty() || ((String)tokens.get(0)).startsWith("-")) {
GlobalWriter.printException(new IllegalArgumentException("Object name query not specified"));
return;
}
StringTokenizer queryTokens = new StringTokenizer((String)tokens.remove(0), COMMAND_OPTION_DELIMETER);
while (queryTokens.hasMoreTokens()) {
queryAddObjects.add(queryTokens.nextToken());
}
}
// If token is a substractive object name query option
else if (token.startsWith("--xobjname")) {
// If no object name query is specified, or next token is a new option
if (tokens.isEmpty() || ((String)tokens.get(0)).startsWith("-")) {
GlobalWriter.printException(new IllegalArgumentException("Object name query not specified"));
return;
}
StringTokenizer queryTokens = new StringTokenizer((String)tokens.remove(0), COMMAND_OPTION_DELIMETER);
while (queryTokens.hasMoreTokens()) {
querySubObjects.add(queryTokens.nextToken());
}
}
// If token is a view option
else if (token.startsWith("--view")) {
// If no view specified, or next token is a new option
if (tokens.isEmpty() || ((String)tokens.get(0)).startsWith("-")) {
GlobalWriter.printException(new IllegalArgumentException("Attributes to view not specified"));
return;
}
// Add the attributes to view
Enumeration viewTokens = new StringTokenizer((String)tokens.remove(0), COMMAND_OPTION_DELIMETER);
while (viewTokens.hasMoreElements()) {
queryViews.add(viewTokens.nextElement());
}
}
// Let super class handle unknown option
else {
super.handleOption(token, tokens);
}
}
/**
* Print the help messages for the browse command
*/
protected void printHelp() {
GlobalWriter.printHelp(helpFile);
}
protected String[] helpFile = new String[] {
"Task Usage: Main query [query-options]",
"Description: Display selected broker component's attributes and statistics.",
"",
"Query Options:",
" -Q<type>=<name> Add to the search list the specific object type matched",
" by the defined object identifier.",
" -xQ<type>=<name> Remove from the search list the specific object type",
" matched by the object identifier.",
" --objname <query> Add to the search list objects matched by the query similar",
" to the JMX object name format.",
" --xobjname <query> Remove from the search list objects matched by the query",
" similar to the JMX object name format.",
" --view <attr1>,<attr2>,... Select the specific attribute of the object to view.",
" By default all attributes will be displayed.",
" --jmxurl <url> Set the JMX URL to connect to.",
" --version Display the version information.",
" -h,-?,--help Display the query broker help information.",
"",
"Examples:",
" Main query",
" - Print all the attributes of all registered objects queues, topics, connections, etc).",
"",
" Main query -QQueue=TEST.FOO",
" - Print all the attributes of the queue with destination name TEST.FOO.",
"",
" Main query -QTopic=*",
" - Print all the attributes of all registered topics.",
"",
" Main query --view EnqueueCount,DequeueCount",
" - Print the attributes EnqueueCount and DequeueCount of all registered objects.",
"",
" Main -QTopic=* --view EnqueueCount,DequeueCount",
" - Print the attributes EnqueueCount and DequeueCount of all registered topics.",
"",
" Main -QTopic=* -QQueue=* --view EnqueueCount,DequeueCount",
" - Print the attributes EnqueueCount and DequeueCount of all registered topics and",
" queues.",
"",
" Main -QTopic=* -xQTopic=ActiveMQ.Advisory.*",
" - Print all attributes of all topics except those that has a name that begins",
" with \"ActiveMQ.Advisory\".",
"",
" Main --objname Type=*Connect*,BrokerName=local* -xQNetworkConnector=*",
" - Print all attributes of all connectors, connections excluding network connectors",
" that belongs to the broker that begins with local.",
"",
" Main -QQueue=* -xQQueue=????",
" - Print all attributes of all queues except those that are 4 letters long.",
"",
};
}

View File

@ -0,0 +1,110 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.command;
import org.apache.activemq.broker.console.formatter.GlobalWriter;
import org.apache.activemq.broker.console.formatter.CommandShellOutputFormatter;
import java.util.List;
import java.util.Arrays;
import java.util.ArrayList;
import java.io.InputStream;
import java.io.PrintStream;
public class ShellCommand extends AbstractCommand {
/**
* Main method to run a command shell client.
* @param args - command line arguments
* @param in - input stream to use
* @param out - output stream to use
* @return 0 for a successful run, -1 if there are any exception
*/
public static int main(String[] args, InputStream in, PrintStream out) {
GlobalWriter.instantiate(new CommandShellOutputFormatter(out));
// Convert arguments to list for easier management
List tokens = new ArrayList(Arrays.asList(args));
ShellCommand main = new ShellCommand();
try {
main.execute(tokens);
return 0;
} catch (Exception e) {
GlobalWriter.printException(e);
return -1;
}
}
/**
* Parses for specific command task, default task is a start task.
* @param tokens - command arguments
* @throws Exception
*/
protected void runTask(List tokens) throws Exception {
// Process task token
if( tokens.size() > 0 ) {
String taskToken = (String)tokens.remove(0);
if (taskToken.equals("start")) {
new StartCommand().execute(tokens);
} else if (taskToken.equals("stop")) {
new ShutdownCommand().execute(tokens);
} else if (taskToken.equals("list")) {
new ListCommand().execute(tokens);
} else if (taskToken.equals("query")) {
new QueryCommand().execute(tokens);
} else if (taskToken.equals("browse")) {
new BrowseCommand().execute(tokens);
} else {
// If not valid task, push back to list
tokens.add(0, taskToken);
new StartCommand().execute(tokens);
}
} else {
new StartCommand().execute(tokens);
}
}
/**
* Print the help messages for the browse command
*/
protected void printHelp() {
GlobalWriter.printHelp(helpFile);
}
protected String[] helpFile = new String[] {
"Usage: Main [--extdir <dir>] [task] [task-options] [task data]",
"",
"Tasks (default task is start):",
" start - Creates and starts a broker using a configuration file, or a broker URI.",
" stop - Stops a running broker specified by the broker name.",
" list - Lists all available brokers in the specified JMX context.",
" query - Display selected broker component's attributes and statistics.",
" browse - Display selected messages in a specified destination.",
"",
"Task Options (Options specific to each task):",
" --extdir <dir> - Add the jar files in the directory to the classpath.",
" --version - Display the version information.",
" -h,-?,--help - Display this help information. To display task specific help, use Main [task] -h,-?,--help",
"",
"Task Data:",
" - Information needed by each specific task.",
""
};
}

View File

@ -14,44 +14,50 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.activemq.broker.console;
package org.apache.activemq.broker.console.command;
import org.apache.activemq.broker.console.JmxMBeansUtil;
import org.apache.activemq.broker.console.formatter.GlobalWriter;
import javax.management.MBeanServerConnection;
import javax.management.ObjectName;
import javax.management.ObjectInstance;
import javax.management.remote.JMXServiceURL;
import java.util.List;
import java.util.Set;
import java.util.Iterator;
import java.util.Collection;
import java.util.HashSet;
public class ShutdownCommand extends AbstractJmxCommand {
private boolean isStopAllBrokers = false;
protected void execute(List brokerNames) {
/**
* Shuts down the specified broker or brokers
* @param brokerNames - names of brokers to shutdown
* @throws Exception
*/
protected void runTask(List brokerNames) throws Exception {
try {
Set mbeans = new HashSet();
MBeanServerConnection server = createJmxConnector().getMBeanServerConnection();
Collection mbeans;
// Stop all brokers
if (isStopAllBrokers) {
mbeans = AmqJmxSupport.getAllBrokers(server);
mbeans = JmxMBeansUtil.getAllBrokers(useJmxServiceUrl());
brokerNames.clear();
}
// Stop the default broker
else if (brokerNames.isEmpty()) {
mbeans = AmqJmxSupport.getAllBrokers(server);
mbeans = JmxMBeansUtil.getAllBrokers(useJmxServiceUrl());
// If there is no broker to stop
if (mbeans.isEmpty()) {
System.out.println("There are no brokers to stop.");
GlobalWriter.printInfo("There are no brokers to stop.");
return;
// There should only be one broker to stop
} else if (mbeans.size() > 1) {
System.out.println("There are multiple brokers to stop. Please select the broker(s) to stop or use --all to stop all brokers.");
System.out.println();
AmqJmxSupport.printBrokerList(mbeans);
GlobalWriter.printInfo("There are multiple brokers to stop. Please select the broker(s) to stop or use --all to stop all brokers.");
return;
// Get the first broker only
@ -65,11 +71,12 @@ public class ShutdownCommand extends AbstractJmxCommand {
// Stop each specified broker
else {
String brokerName;
mbeans = new HashSet();
while (!brokerNames.isEmpty()) {
brokerName = (String)brokerNames.remove(0);
Set matchedBrokers = AmqJmxSupport.getBrokers(server, brokerName);
Collection matchedBrokers = JmxMBeansUtil.getBrokersByName(useJmxServiceUrl(), brokerName);
if (matchedBrokers.isEmpty()) {
System.out.println(brokerName + " did not match any running brokers.");
GlobalWriter.printInfo(brokerName + " did not match any running brokers.");
} else {
mbeans.addAll(matchedBrokers);
}
@ -77,49 +84,47 @@ public class ShutdownCommand extends AbstractJmxCommand {
}
// Stop all brokers in set
stopBrokers(server, mbeans);
closeJmxConnector();
} catch (Throwable e) {
System.out.println("Failed to execute stop task. Reason: " + e);
stopBrokers(useJmxServiceUrl(), mbeans);
} catch (Exception e) {
GlobalWriter.printException(new RuntimeException("Failed to execute stop task. Reason: " + e));
throw new Exception(e);
}
}
protected void stopBrokers(MBeanServerConnection server, Set brokerBeans) throws Exception {
/**
* Stops the list of brokers.
* @param jmxServiceUrl - JMX service url to connect to
* @param brokerBeans - broker mbeans to stop
* @throws Exception
*/
protected void stopBrokers(JMXServiceURL jmxServiceUrl, Collection brokerBeans) throws Exception {
MBeanServerConnection server = createJmxConnector().getMBeanServerConnection();
ObjectName brokerObjName;
for (Iterator i=brokerBeans.iterator(); i.hasNext();) {
brokerObjName = ((ObjectInstance)i.next()).getObjectName();
String brokerName = brokerObjName.getKeyProperty("BrokerName");
System.out.println("Stopping broker: " + brokerName);
GlobalWriter.print("Stopping broker: " + brokerName);
try {
server.invoke(brokerObjName, "terminateJVM", new Object[] {new Integer(0)}, new String[] {"int"});
System.out.println("Succesfully stopped broker: " + brokerName);
GlobalWriter.print("Succesfully stopped broker: " + brokerName);
} catch (Exception e) {
// TODO: Check exceptions throwned
//System.out.println("Failed to stop broker: [ " + brokerName + " ]. Reason: " + e.getMessage());
}
}
closeJmxConnector();
}
protected void printHelp() {
System.out.println("Task Usage: Main stop [stop-options] [broker-name1] [broker-name2] ...");
System.out.println("Description: Stops a running broker.");
System.out.println("");
System.out.println("Stop Options:");
System.out.println(" --jmxurl <url> Set the JMX URL to connect to.");
System.out.println(" --all Stop all brokers.");
System.out.println(" --version Display the version information.");
System.out.println(" -h,-?,--help Display the stop broker help information.");
System.out.println("");
System.out.println("Broker Names:");
System.out.println(" Name of the brokers that will be stopped.");
System.out.println(" If omitted, it is assumed that there is only one broker running, and it will be stopped.");
System.out.println(" Use -all to stop all running brokers.");
System.out.println("");
}
/**
* Handle the --all option.
* @param token - option token to handle
* @param tokens - succeeding command arguments
* @throws Exception
*/
protected void handleOption(String token, List tokens) throws Exception {
// Try to handle the options first
if (token.equals("--all")) {
@ -129,4 +134,28 @@ public class ShutdownCommand extends AbstractJmxCommand {
super.handleOption(token, tokens);
}
}
/**
* Print the help messages for the browse command
*/
protected void printHelp() {
GlobalWriter.printHelp(helpFile);
}
protected String[] helpFile = new String[] {
"Task Usage: Main stop [stop-options] [broker-name1] [broker-name2] ...",
"Description: Stops a running broker.",
"",
"Stop Options:",
" --jmxurl <url> Set the JMX URL to connect to.",
" --all Stop all brokers.",
" --version Display the version information.",
" -h,-?,--help Display the stop broker help information.",
"",
"Broker Names:",
" Name of the brokers that will be stopped.",
" If omitted, it is assumed that there is only one broker running, and it will be stopped.",
" Use -all to stop all running brokers.",
""
};
}

View File

@ -15,10 +15,11 @@
* limitations under the License.
*/
package org.apache.activemq.broker.console;
package org.apache.activemq.broker.console.command;
import org.apache.activemq.broker.BrokerFactory;
import org.apache.activemq.broker.BrokerService;
import org.apache.activemq.broker.console.formatter.GlobalWriter;
import java.util.List;
import java.util.ArrayList;
@ -37,7 +38,7 @@ public class StartCommand extends AbstractCommand {
* The default task to start a broker or a group of brokers
* @param brokerURIs
*/
protected void execute(List brokerURIs) {
protected void runTask(List brokerURIs) throws Exception {
try {
// If no config uri, use default setting
if (brokerURIs.isEmpty()) {
@ -54,7 +55,7 @@ public class StartCommand extends AbstractCommand {
try {
setConfigUri(new URI(strConfigURI));
} catch (URISyntaxException e) {
printError("Invalid broker configuration URI: " + strConfigURI + ", reason: " + e.getMessage());
GlobalWriter.printException(e);
return;
}
@ -64,8 +65,9 @@ public class StartCommand extends AbstractCommand {
// Prevent the main thread from exiting unless it is terminated elsewhere
waitForShutdown();
} catch (Throwable e) {
System.out.println("Failed to execute start task. Reason: " + e);
} catch (Exception e) {
GlobalWriter.printException(new RuntimeException("Failed to execute start task. Reason: " + e));
throw new Exception(e);
}
}
@ -114,37 +116,6 @@ public class StartCommand extends AbstractCommand {
}
}
/**
* Prints the help for the start broker task
*/
protected void printHelp() {
System.out.println("Task Usage: Main start [start-options] [uri]");
System.out.println("Description: Creates and starts a broker using a configuration file, or a broker URI.");
System.out.println("");
System.out.println("Start Options:");
System.out.println(" --extdir <dir> Add the jar files in the directory to the classpath.");
System.out.println(" -D<name>=<value> Define a system property.");
System.out.println(" --version Display the version information.");
System.out.println(" -h,-?,--help Display the start broker help information.");
System.out.println("");
System.out.println("URI:");
System.out.println("");
System.out.println(" XBean based broker configuration:");
System.out.println("");
System.out.println(" Example: Main xbean:file:activemq.xml");
System.out.println(" Loads the xbean configuration file from the current working directory");
System.out.println(" Example: Main xbean:activemq.xml");
System.out.println(" Loads the xbean configuration file from the classpath");
System.out.println("");
System.out.println(" URI Parameter based broker configuration:");
System.out.println("");
System.out.println(" Example: Main broker:(tcp://localhost:61616, tcp://localhost:5000)?useJmx=true");
System.out.println(" Configures the broker with 2 transport connectors and jmx enabled");
System.out.println(" Example: Main broker:(tcp://localhost:61616, network:tcp://localhost:5000)?persistent=false");
System.out.println(" Configures the broker with 1 transport connector, and 1 network connector and persistence disabled");
System.out.println("");
}
/**
* Sets the current configuration URI used by the start task
* @param uri
@ -160,4 +131,38 @@ public class StartCommand extends AbstractCommand {
public URI getConfigUri() {
return configURI;
}
/**
* Print the help messages for the browse command
*/
protected void printHelp() {
GlobalWriter.printHelp(helpFile);
}
protected String[] helpFile = new String[] {
"Task Usage: Main start [start-options] [uri]",
"Description: Creates and starts a broker using a configuration file, or a broker URI.",
"",
"Start Options:",
" -D<name>=<value> Define a system property.",
" --version Display the version information.",
" -h,-?,--help Display the start broker help information.",
"",
"URI:",
"",
" XBean based broker configuration:",
"",
" Example: Main xbean:file:activemq.xml",
" Loads the xbean configuration file from the current working directory",
" Example: Main xbean:activemq.xml",
" Loads the xbean configuration file from the classpath",
"",
" URI Parameter based broker configuration:",
"",
" Example: Main broker:(tcp://localhost:61616, tcp://localhost:5000)?useJmx=true",
" Configures the broker with 2 transport connectors and jmx enabled",
" Example: Main broker:(tcp://localhost:61616, network:tcp://localhost:5000)?persistent=false",
" Configures the broker with 1 transport connector, and 1 network connector and persistence disabled",
""
};
}

View File

@ -0,0 +1,45 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import java.util.List;
import java.util.StringTokenizer;
import java.util.Collections;
public abstract class AbstractQueryFilter implements QueryFilter {
protected QueryFilter next;
/**
* Creates a query filter, with the next filter specified by next.
* @param next - the next query filter
*/
protected AbstractQueryFilter(QueryFilter next) {
this.next = next;
}
/**
* Performs a query given the query string
* @param query - query string
* @return objects that matches the query
* @throws Exception
*/
public List query(String query) throws Exception {
// Converts string query to map query
StringTokenizer tokens = new StringTokenizer(query, QUERY_DELIMETER);
return query(Collections.list(tokens));
}
}

View File

@ -0,0 +1,79 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import java.util.Set;
import java.util.Map;
import java.util.HashMap;
import java.util.Iterator;
public class GroupPropertiesViewFilter extends PropertiesViewFilter {
/**
* Creates a group properties filter that is able to filter the display result based on a group prefix
* @param next - the next query filter
*/
public GroupPropertiesViewFilter(QueryFilter next) {
super(next);
}
/**
* Creates a group properties filter that is able to filter the display result based on a group prefix
* @param groupView - the group filter to use
* @param next - the next query filter
*/
public GroupPropertiesViewFilter(Set groupView, QueryFilter next) {
super(groupView, next);
}
/**
* Filter the properties that matches the group prefix only.
* @param data - map data to filter
* @return - filtered map data
*/
protected Map filterView(Map data) {
// If no view specified, display all attributes
if (viewFilter == null || viewFilter.isEmpty()) {
return data;
}
Map newData;
try {
// Lets try to use the same class as the original
newData = (Map)data.getClass().newInstance();
} catch (Exception e) {
// Lets use a default HashMap
newData = new HashMap();
}
// Filter the keys to view
for (Iterator i=data.keySet().iterator(); i.hasNext();) {
String key = (String)i.next();
// Checks if key matches any of the group filter
for (Iterator j=viewFilter.iterator(); j.hasNext();) {
String group = (String)j.next();
if (key.startsWith(group)) {
newData.put(key, data.get(key));
break;
}
}
}
return newData;
}
}

View File

@ -0,0 +1,148 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import javax.management.remote.JMXServiceURL;
import javax.management.remote.JMXConnectorFactory;
import javax.management.remote.JMXConnector;
import javax.management.ObjectInstance;
import javax.management.ObjectName;
import javax.management.MBeanServerConnection;
import javax.management.ReflectionException;
import javax.management.InstanceNotFoundException;
import javax.management.AttributeList;
import javax.management.Attribute;
import javax.management.MBeanAttributeInfo;
import javax.management.IntrospectionException;
import java.util.Set;
import java.util.Collection;
import java.util.Iterator;
import java.util.ArrayList;
import java.util.List;
import java.io.IOException;
public class MBeansAttributeQueryFilter extends AbstractQueryFilter {
public static final String KEY_OBJECT_NAME_ATTRIBUTE = "Attribute:ObjectName:";
private JMXServiceURL jmxServiceUrl;
private Set attribView;
/**
* Create an mbean attributes query filter that is able to select specific mbean attributes based on the object name to get.
* @param jmxServiceUrl - JMX service url to connect to.
* @param attribView - the attributes to extract
* @param next - the next query filter
*/
public MBeansAttributeQueryFilter(JMXServiceURL jmxServiceUrl, Set attribView, MBeansObjectNameQueryFilter next) {
super(next);
this.jmxServiceUrl = jmxServiceUrl;
this.attribView = attribView;
}
/**
* Filter the query by retrieving the attributes specified, this will modify the collection to a list of AttributeList
* @param queries - query list
* @return List of AttributeList, which includes the ObjectName, which has a key of MBeansAttributeQueryFilter.KEY_OBJECT_NAME_ATTRIBUTE
* @throws Exception
*/
public List query(List queries) throws Exception {
return getMBeanAttributesCollection(next.query(queries));
}
/**
* Retrieve the specified attributes of the mbean
* @param result - collection of ObjectInstances and/or ObjectNames
* @return List of AttributeList
* @throws IOException
* @throws ReflectionException
* @throws InstanceNotFoundException
* @throws NoSuchMethodException
*/
protected List getMBeanAttributesCollection(Collection result) throws IOException, ReflectionException, InstanceNotFoundException, NoSuchMethodException, IntrospectionException {
List mbeansCollection = new ArrayList();
for (Iterator i=result.iterator(); i.hasNext();) {
Object mbean = i.next();
if (mbean instanceof ObjectInstance) {
mbeansCollection.add(getMBeanAttributes(((ObjectInstance)mbean).getObjectName(), attribView));
} else if (mbean instanceof ObjectName) {
mbeansCollection.add(getMBeanAttributes((ObjectName)mbean, attribView));
} else {
throw new NoSuchMethodException("Cannot get the mbean attributes for class: " + mbean.getClass().getName());
}
}
return mbeansCollection;
}
/**
* Retrieve the specified attributes of the mbean
* @param obj - mbean ObjectInstance
* @param attrView - list of attributes to retrieve
* @return AttributeList for the mbean
* @throws ReflectionException
* @throws InstanceNotFoundException
* @throws IOException
*/
protected AttributeList getMBeanAttributes(ObjectInstance obj, Set attrView) throws ReflectionException, InstanceNotFoundException, IOException, IntrospectionException {
return getMBeanAttributes(obj.getObjectName(), attrView);
}
/**
* Retrieve the specified attributes of the mbean
* @param objName - mbean ObjectName
* @param attrView - list of attributes to retrieve
* @return AttributeList for the mbean
* @throws IOException
* @throws ReflectionException
* @throws InstanceNotFoundException
*/
protected AttributeList getMBeanAttributes(ObjectName objName, Set attrView) throws IOException, ReflectionException, InstanceNotFoundException, IntrospectionException {
JMXConnector jmxConnector = JMXConnectorFactory.connect(jmxServiceUrl);
MBeanServerConnection server = jmxConnector.getMBeanServerConnection();
// If no attribute view specified, get all attributes
String[] attribs;
if (attrView == null || attrView.isEmpty()) {
MBeanAttributeInfo[] infos = server.getMBeanInfo(objName).getAttributes();
attribs = new String[infos.length];
for (int i=0; i<infos.length; i++) {
if (infos[i].isReadable()) {
attribs[i] = infos[i].getName();
}
}
// Get selected attributes
} else {
attribs = new String[attrView.size()];
int count = 0;
for (Iterator i=attrView.iterator(); i.hasNext();) {
attribs[count++] = (String)i.next();
}
}
AttributeList attribList = server.getAttributes(objName, attribs);
jmxConnector.close();
attribList.add(0, new Attribute(KEY_OBJECT_NAME_ATTRIBUTE, objName));
return attribList;
}
}

View File

@ -0,0 +1,161 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import javax.management.remote.JMXServiceURL;
import javax.management.remote.JMXConnector;
import javax.management.remote.JMXConnectorFactory;
import javax.management.ObjectName;
import javax.management.MalformedObjectNameException;
import javax.management.MBeanServerConnection;
import javax.management.QueryExp;
import java.util.Iterator;
import java.util.List;
import java.util.ArrayList;
import java.net.MalformedURLException;
import java.io.IOException;
public class MBeansObjectNameQueryFilter extends AbstractQueryFilter {
public static final String DEFAULT_JMX_DOMAIN = "org.apache.activemq";
public static final String QUERY_EXP_PREFIX = "MBeans.QueryExp.";
private JMXServiceURL jmxServiceUrl;
/**
* Creates an mbeans object name query filter that will query on the given JMX Service URL
* @param jmxUrl - JMX service URL to connect to
* @throws MalformedURLException
*/
public MBeansObjectNameQueryFilter(String jmxUrl) throws MalformedURLException {
this(new JMXServiceURL(jmxUrl));
}
/**
* Creates an mbeans objecet name query filter that will query on the given JMX Service URL
* @param jmxUrl - JMX service URL to connect to
*/
public MBeansObjectNameQueryFilter(JMXServiceURL jmxUrl) {
super(null);
this.jmxServiceUrl = jmxUrl;
}
/**
* Queries the JMX service using a mapping of keys and values to construct the object name
* @param queries - mapping of keys and values
* @return collection of ObjectInstance that matches the query
* @throws MalformedObjectNameException - if the given string is an invalid object name
* @throws IOException - if there is a problem querying the JMX context
*/
public List query(List queries) throws MalformedObjectNameException, IOException {
// Query all mbeans
if (queries == null || queries.isEmpty()) {
return queryMBeans(new ObjectName(DEFAULT_JMX_DOMAIN + ":*"), null);
}
// Constructs object name query
String objNameQuery = "";
String queryExp = "";
for (Iterator i=queries.iterator(); i.hasNext();) {
String key = (String)i.next();
String val = "";
int pos = key.indexOf("=");
if (pos >= 0) {
val = key.substring(pos + 1);
key = key.substring(0, pos);
}
if (val.startsWith(QUERY_EXP_PREFIX)) {
// do nothing as of the moment
} else if (key != "" && val != "") {
objNameQuery = objNameQuery + key + "=" + val + ",";
}
}
// Append * to object name
objNameQuery = objNameQuery + "*";
return queryMBeans(new ObjectName(DEFAULT_JMX_DOMAIN + ":" + objNameQuery), queryExp);
}
/**
* Advance query that enables you to specify both the object name and the query expression to use.
* Note: Query expression is currently unsupported.
* @param objName - object name to use for query
* @param queryExpStr - query expression string
* @return set of mbeans that matches the query
* @throws IOException - if there is a problem querying the JMX context
*/
protected List queryMBeans(ObjectName objName, String queryExpStr) throws IOException {
JMXConnector jmxConn = createJmxConnector();
MBeanServerConnection server = jmxConn.getMBeanServerConnection();
QueryExp queryExp = createQueryExp(queryExpStr);
// Convert mbeans set to list to make it standard throughout the query filter
List mbeans = new ArrayList(server.queryMBeans(objName, queryExp));
jmxConn.close();
return mbeans;
}
/**
* Get the JMX service URL the query is connecting to.
* @return JMX service URL
*/
public JMXServiceURL getJmxServiceUrl() {
return jmxServiceUrl;
}
/**
* Sets the JMX service URL the query is going to connect to.
* @param jmxServiceUrl - new JMX service URL
*/
public void setJmxServiceUrl(JMXServiceURL jmxServiceUrl) {
this.jmxServiceUrl = jmxServiceUrl;
}
/**
* Sets the JMX service URL the query is going to connect to.
* @param jmxServiceUrl - new JMX service URL
*/
public void setJmxServiceUrl(String jmxServiceUrl) throws MalformedURLException {
setJmxServiceUrl(new JMXServiceURL(jmxServiceUrl));
}
/**
* Creates a JMX connector
* @return JMX connector
* @throws IOException
*/
protected JMXConnector createJmxConnector() throws IOException {
return JMXConnectorFactory.connect(getJmxServiceUrl());
}
/**
* Creates a query expression based on the query expression string
* Note: currently unsupported
* @param queryExpStr - query expression string
* @return the created query expression
*/
protected QueryExp createQueryExp(String queryExpStr) {
// Currently unsupported
return null;
}
}

View File

@ -0,0 +1,127 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import javax.management.ObjectInstance;
import javax.management.ObjectName;
import javax.management.AttributeList;
import javax.management.Attribute;
import java.util.Map;
import java.util.Iterator;
import java.util.regex.Pattern;
import java.lang.reflect.Method;
public class MBeansRegExQueryFilter extends RegExQueryFilter {
/**
* Creates a regular expression query that is able to match the values of specific mbeans
* @param next - next query filter
*/
public MBeansRegExQueryFilter(QueryFilter next) {
super(next);
}
/**
* Try to match the object data using the regular expression map. The regex map contains a key-value mapping of an attribute
* key to a regular expression the value of the key should match. The basic rule of matching is that the data must contain
* a property key that is included in the regex map, and that the value of the property key should match the regex specified.
* @param data - object data to match
* @param regex - regex map
* @return true if the data matches the regex map specified
* @throws Exception
*/
protected boolean matches(Object data, Map regex) throws Exception {
// Use reflection to determine where the object should go
try {
Method method = this.getClass().getDeclaredMethod("matches", new Class[] {data.getClass(), Map.class});
return ((Boolean)method.invoke(this, new Object[] {data, regex})).booleanValue();
} catch (NoSuchMethodException e) {
return false;
}
}
/**
* Try to match the object instance using the regular expression map
* @param data - object instance to match
* @param regex - regex map
* @return true if the object instance matches the regex map
*/
protected boolean matches(ObjectInstance data, Map regex) {
return matches(data.getObjectName(), regex);
}
/**
* Try to match the object name instance using the regular expression map
* @param data - object name to match
* @param regex - regex map
* @return true if the object name matches the regex map
*/
protected boolean matches(ObjectName data, Map regex) {
for (Iterator i=regex.keySet().iterator(); i.hasNext();) {
String key = (String)i.next();
String target = data.getKeyProperty(key);
// Try to match the value of the property of the object name
if (target != null && !((Pattern)regex.get(key)).matcher(target).matches()) {
return false;
}
}
return true;
}
/**
* Try to match the attribute list using teh regular expression map
* @param data - attribute list to match
* @param regex - regex map
* @return true if the attribute list matches the regex map
*/
protected boolean matches(AttributeList data, Map regex) {
for (Iterator i=regex.keySet().iterator(); i.hasNext();) {
String key = (String)i.next();
// Try to match each regex to the attributes of the mbean including its ObjectName
for (Iterator j=data.iterator(); j.hasNext();) {
Attribute attrib = (Attribute)j.next();
// Try to match to the properties of the ObjectName
if (attrib.getName().equals(MBeansAttributeQueryFilter.KEY_OBJECT_NAME_ATTRIBUTE)) {
String target = ((ObjectName)attrib.getValue()).getKeyProperty(key);
if (target == null || !((Pattern)regex.get(key)).matcher(target).matches()) {
return false;
} else {
// If match skip to the next regex
break;
}
// Try to match to the mbean attributes
} else if (attrib.getName().equals(key)) {
if (!((Pattern)regex.get(key)).matcher(attrib.getValue().toString()).matches()) {
return false;
} else {
// If match skip to the next regex
break;
}
// If mbean does not contain the specified attribute
} else {
return false;
}
}
}
return true;
}
}

View File

@ -0,0 +1,259 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import org.apache.activemq.broker.console.formatter.GlobalWriter;
import org.apache.activemq.broker.console.AmqMessagesUtil;
import org.apache.activemq.command.ActiveMQDestination;
import org.apache.activemq.command.ActiveMQObjectMessage;
import org.apache.activemq.command.ActiveMQBytesMessage;
import org.apache.activemq.command.ActiveMQTextMessage;
import org.apache.activemq.command.ActiveMQMapMessage;
import org.apache.activemq.command.ActiveMQStreamMessage;
import org.apache.activemq.command.ActiveMQMessage;
import javax.management.ObjectInstance;
import javax.management.ObjectName;
import javax.management.AttributeList;
import javax.management.Attribute;
import javax.jms.JMSException;
import javax.jms.DeliveryMode;
import java.util.Map;
import java.util.Properties;
import java.util.Iterator;
import java.util.Enumeration;
import java.lang.reflect.Method;
public class MapTransformFilter extends ResultTransformFilter {
/**
* Creates a Map transform filter that is able to transform a variety of objects to a properties map object
* @param next - the next query filter
*/
public MapTransformFilter(QueryFilter next) {
super(next);
}
/**
* Transform the given object to a Map object
* @param object - object to transform
* @return map object
*/
protected Object transformElement(Object object) throws Exception {
// Use reflection to determine how the object should be transformed
try {
Method method = this.getClass().getDeclaredMethod("transformToMap", new Class[] {object.getClass()});
return (Map)method.invoke(this, new Object[] {object});
} catch (NoSuchMethodException e) {
GlobalWriter.print("Unable to transform mbean of type: " + object.getClass().getName() + ". No corresponding transformToMap method found.");
return null;
}
}
/**
* Transform an ObjectInstance mbean to a Map
* @param obj - ObjectInstance format of an mbean
* @return map object
*/
protected Map transformToMap(ObjectInstance obj) {
return transformToMap(obj.getObjectName());
}
/**
* Transform an ObjectName mbean to a Map
* @param objname - ObjectName format of an mbean
* @return map object
*/
protected Map transformToMap(ObjectName objname) {
Properties props = new Properties();
// Parse object properties
Map objProps = objname.getKeyPropertyList();
for (Iterator i=objProps.keySet().iterator(); i.hasNext();) {
Object key = i.next();
Object val = objProps.get(key);
if (val != null) {
props.setProperty(key.toString(), val.toString());
}
}
return props;
}
/**
* Transform an Attribute List format of an mbean to a Map
* @param list - AttributeList format of an mbean
* @return map object
*/
protected Map transformToMap(AttributeList list) {
Properties props = new Properties();
for (Iterator i=list.iterator(); i.hasNext();) {
Attribute attrib = (Attribute)i.next();
// If attribute is an ObjectName
if (attrib.getName().equals(MBeansAttributeQueryFilter.KEY_OBJECT_NAME_ATTRIBUTE)) {
props.putAll(transformToMap((ObjectName)attrib.getValue()));
} else {
if (attrib.getValue() != null) {
props.setProperty(attrib.getName(), attrib.getValue().toString());
}
}
}
return props;
}
/**
* Transform an ActiveMQTextMessage to a Map
* @param msg - text message to trasnform
* @return map object
* @throws JMSException
*/
protected Map transformToMap(ActiveMQTextMessage msg) throws JMSException {
Properties props = new Properties();
props.putAll(transformToMap((ActiveMQMessage)msg));
if (msg.getText() != null) {
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_BODY_PREFIX + "JMSText", msg.getText());
}
return props;
}
/**
* Transform an ActiveMQBytesMessage to a Map
* @param msg - bytes message to transform
* @return map object
* @throws JMSException
*/
protected Map transformToMap(ActiveMQBytesMessage msg) throws JMSException {
Properties props = new Properties();
props.putAll(transformToMap((ActiveMQMessage)msg));
long bodyLength = msg.getBodyLength();
byte[] msgBody;
int i=0;
// Create separate bytes messages
for (i=0; i<(bodyLength/Integer.MAX_VALUE); i++) {
msgBody = new byte[Integer.MAX_VALUE];
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_BODY_PREFIX + "JMSBytes:" + (i+1), new String(msgBody));
}
msgBody = new byte[(int)(bodyLength % Integer.MAX_VALUE)];
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_BODY_PREFIX + "JMSBytes:" + (i+1), new String(msgBody));
return props;
}
/**
* Transform an ActiveMQMessage to a Map
* @param msg - object message to transform
* @return map object
* @throws JMSException
*/
protected Map transformToMap(ActiveMQObjectMessage msg) throws JMSException {
Properties props = new Properties();
props.putAll(transformToMap((ActiveMQMessage)msg));
if (msg.getObject() != null) {
// Just add the class name and toString value of the object
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_BODY_PREFIX + "JMSObjectClass", msg.getObject().getClass().getName());
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_BODY_PREFIX + "JMSObjectString", msg.getObject().toString());
}
return props;
}
/**
* Transform an ActiveMQMapMessage to a Map
* @param msg - map message to transform
* @return map object
* @throws JMSException
*/
protected Map transformToMap(ActiveMQMapMessage msg) throws JMSException {
Properties props = new Properties();
props.putAll(transformToMap((ActiveMQMessage)msg));
// Get map properties
Enumeration e = msg.getMapNames();
while (e.hasMoreElements()) {
String key = (String)e.nextElement();
Object val = msg.getObject(key);
if (val != null) {
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_BODY_PREFIX + key, val.toString());
}
}
return props;
}
/**
* Transform an ActiveMQStreamMessage to a Map
* @param msg - stream message to transform
* @return map object
* @throws JMSException
*/
protected Map transformToMap(ActiveMQStreamMessage msg) throws JMSException {
Properties props = new Properties();
props.putAll(transformToMap((ActiveMQMessage)msg));
// Just set the toString of the message as the body of the stream message
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_BODY_PREFIX + "JMSStreamMessage", msg.toString());
return props;
}
/**
* Transform an ActiveMQMessage to a Map
* @param msg - message to transform
* @return map object
* @throws JMSException
*/
protected Map transformToMap(ActiveMQMessage msg) throws JMSException {
Properties props = new Properties();
// Get JMS properties
if (msg.getJMSCorrelationID() != null) {
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX + "JMSCorrelationID", msg.getJMSCorrelationID());
}
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX + "JMSDeliveryMode", (msg.getJMSDeliveryMode()==DeliveryMode.PERSISTENT) ? "persistent" : "non-persistent");
if (msg.getJMSDestination() != null) {
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX + "JMSDestination", ((ActiveMQDestination)msg.getJMSDestination()).getPhysicalName());
}
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX + "JMSExpiration", Long.toString(msg.getJMSExpiration()));
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX + "JMSMessageID", msg.getJMSMessageID());
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX + "JMSPriority", Integer.toString(msg.getJMSPriority()));
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX + "JMSRedelivered", Boolean.toString(msg.getJMSRedelivered()));
if (msg.getJMSReplyTo() != null) {
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX + "JMSReplyTo", ((ActiveMQDestination)msg.getJMSReplyTo()).getPhysicalName());
}
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX + "JMSTimestamp", Long.toString(msg.getJMSTimestamp()));
if (msg.getJMSType() != null) {
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_HEADER_PREFIX + "JMSType", msg.getJMSType());
}
// Get custom properties
Enumeration e = msg.getPropertyNames();
while (e.hasMoreElements()) {
String name = (String)e.nextElement();
if (msg.getObjectProperty(name) != null) {
props.setProperty(AmqMessagesUtil.JMS_MESSAGE_CUSTOM_PREFIX + name, msg.getObjectProperty(name).toString());
}
}
return props;
}
}

View File

@ -0,0 +1,152 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import org.apache.activemq.command.ActiveMQTopic;
import org.apache.activemq.command.ActiveMQQueue;
import org.apache.activemq.ActiveMQConnectionFactory;
import javax.jms.Destination;
import javax.jms.Connection;
import javax.jms.JMSException;
import javax.jms.Session;
import javax.jms.QueueBrowser;
import java.net.URI;
import java.util.Collections;
import java.util.List;
import java.util.Iterator;
public class MessagesQueryFilter extends AbstractQueryFilter {
private URI brokerUrl;
private Destination destination;
/**
* Create a JMS message query filter
* @param brokerUrl - broker url to connect to
* @param destination - JMS destination to query
*/
public MessagesQueryFilter(URI brokerUrl, Destination destination) {
super(null);
this.brokerUrl = brokerUrl;
this.destination = destination;
}
/**
* Queries the specified destination using the message selector format query
* @param queries - message selector queries
* @return list messages that matches the selector
* @throws Exception
*/
public List query(List queries) throws Exception {
String selector = "";
// Convert to message selector
for (Iterator i=queries.iterator(); i.hasNext();) {
selector = selector + "(" + i.next().toString() + ") AND ";
}
// Remove last AND
if (selector != "") {
selector = selector.substring(0, selector.length() - 5);
}
if (destination instanceof ActiveMQQueue) {
return queryMessages((ActiveMQQueue)destination, selector);
} else {
return queryMessages((ActiveMQTopic)destination, selector);
}
}
/**
* Query the messages of a queue destination using a queue browser
* @param queue - queue destination
* @param selector - message selector
* @return list of messages that matches the selector
* @throws Exception
*/
protected List queryMessages(ActiveMQQueue queue, String selector) throws Exception {
Connection conn = createConnection(getBrokerUrl());
Session sess = conn.createSession(false, Session.AUTO_ACKNOWLEDGE);
QueueBrowser browser = sess.createBrowser(queue, selector);
List messages = Collections.list(browser.getEnumeration());
conn.close();
return messages;
}
/**
* Query the messages of a topic destination using a message consumer
* @param topic - topic destination
* @param selector - message selector
* @return list of messages that matches the selector
* @throws Exception
*/
protected List queryMessages(ActiveMQTopic topic, String selector) throws Exception {
// TODO: should we use a durable subscriber or a retroactive non-durable subscriber?
// TODO: if a durable subscriber is used, how do we manage it? subscribe/unsubscribe tasks?
return null;
}
/**
* Create and start a JMS connection
* @param brokerUrl - broker url to connect to.
* @return JMS connection
* @throws JMSException
*/
protected Connection createConnection(URI brokerUrl) throws JMSException {
Connection conn = (new ActiveMQConnectionFactory(brokerUrl)).createConnection();
conn.start();
return conn;
}
/**
* Get the broker url being used.
* @return broker url
*/
public URI getBrokerUrl() {
return brokerUrl;
}
/**
* Set the broker url to use.
* @param brokerUrl - broker url
*/
public void setBrokerUrl(URI brokerUrl) {
this.brokerUrl = brokerUrl;
}
/**
* Get the destination being used.
* @return - JMS destination
*/
public Destination getDestination() {
return destination;
}
/**
* Set the destination to use.
* @param destination - JMS destination
*/
public void setDestination(Destination destination) {
this.destination = destination;
}
}

View File

@ -0,0 +1,118 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import java.util.Set;
import java.util.Iterator;
import java.util.Map;
import java.util.HashMap;
import java.util.Collection;
import java.util.ArrayList;
import java.util.List;
public class PropertiesViewFilter implements QueryFilter {
protected QueryFilter next;
protected Set viewFilter;
/**
* Creates a filter that will select the properties of a map object to view
* @param next - the next query filter that will return a collection of maps
*/
public PropertiesViewFilter(QueryFilter next) {
this(null, next);
}
/**
* Creates a filter that will select the properties of a map object to view
* @param viewFilter - the properties to view
* @param next - the next query filter that will return a collection of maps
*/
public PropertiesViewFilter(Set viewFilter, QueryFilter next) {
this.next = next;
this.viewFilter = viewFilter;
}
/**
* Filter the properties to view of the query result
* @param query - the query string
* @return list of objects that has been view filtered
*/
public List query(String query) throws Exception {
return filterViewCollection(next.query(query), viewFilter);
}
/**
* Filter the properties to view of the query result
* @param queries - the query map
* @return list of objects that has been view filtered
* @throws Exception
*/
public List query(List queries) throws Exception {
return filterViewCollection(next.query(queries), viewFilter);
}
/**
* Filter the view of each element in the collection
* @param result - the lists to filter the view from
* @param viewFilter - the views to select
* @return lsit of objects whose view has been filtered
*/
protected List filterViewCollection(Collection result, Set viewFilter) {
// Use a list to allow duplicate entries
List newCollection = new ArrayList();
for (Iterator i=result.iterator(); i.hasNext();) {
newCollection.add(filterView((Map)i.next()));
}
return newCollection;
}
/**
* Select only the attributes to view from the map data
* @param data - data to filter the view from
* @return - data with the view filtered
*/
protected Map filterView(Map data) {
// If no view specified, display all attributes
if (viewFilter == null || viewFilter.isEmpty()) {
return data;
}
Map newData;
try {
// Lets try to use the same class as the original
newData = (Map)data.getClass().newInstance();
} catch (Exception e) {
// Lets use a default HashMap
newData = new HashMap();
}
// Filter the keys to view
for (Iterator i=viewFilter.iterator(); i.hasNext();) {
Object key = i.next();
Object val = data.get(key);
if (val != null) {
newData.put(key, val);
}
}
return newData;
}
}

View File

@ -0,0 +1,39 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import java.util.List;
public interface QueryFilter {
public static final String QUERY_DELIMETER = ",";
/**
* Interface for querying
* @param queryStr - the query string
* @return collection of objects that satisfies the query
* @throws Exception
*/
public List query(String queryStr) throws Exception;
/**
* Interface for querying
* @param queries - list of individual queries
* @return collection of objects that satisfies the query
* @throws Exception
*/
public List query(List queries) throws Exception;
}

View File

@ -0,0 +1,128 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import java.util.regex.Pattern;
import java.util.List;
import java.util.Map;
import java.util.HashMap;
import java.util.ArrayList;
import java.util.Iterator;
public abstract class RegExQueryFilter extends AbstractQueryFilter {
public static final String REGEX_PREFIX = "REGEX:QUERY:";
/**
* Creates a regular expression query that is able to match an object using key-value pattern regex filtering
* @param next
*/
protected RegExQueryFilter(QueryFilter next) {
super(next);
}
/**
* Separates the regular expressions queries from the usual queries. A query is a regex query, if it is key-value pair
* with the format <key>=<value>, and value is a pattern that satisfies the isRegularExpression method.
* @param queries - list of queries
* @return filtered objects that matches the regex query
* @throws Exception
*/
public List query(List queries) throws Exception {
Map regex = new HashMap();
List newQueries = new ArrayList();
// Lets parse for regular expression queries
for (Iterator i=queries.iterator(); i.hasNext();) {
// Get key-value pair
String token = (String)i.next();
String key = "";
String val = "";
int pos = token.indexOf("=");
if (pos >= 0) {
val = token.substring(pos + 1);
key = token.substring(0, pos);
}
// Add the regex query to list and make it a non-factor in the succeeding queries
if (isRegularExpression(val)) {
regex.put(key, compileQuery(val));
// Add the normal query to the query list
} else {
newQueries.add(token);
}
}
// Filter the result using the regular expressions specified
return filterCollectionUsingRegEx(regex, next.query(newQueries));
}
/**
* Checks if a given string is a regular expression query. Currently, a pattern is a regex query, if it starts with
* the RegExQueryFilter.REGEX_PREFIX.
* @param query
* @return
*/
protected boolean isRegularExpression(String query) {
return query.startsWith(REGEX_PREFIX);
}
/**
* Compiles the regex query to a pattern.
* @param query - query string to compile
* @return regex pattern
*/
protected Pattern compileQuery(String query) {
return Pattern.compile(query.substring(REGEX_PREFIX.length()));
}
/**
* Filter the specified colleciton using the regex patterns extracted.
* @param regex - regex map
* @param data - list of objects to filter
* @return filtered list of objects that matches the regex map
* @throws Exception
*/
protected List filterCollectionUsingRegEx(Map regex, List data) throws Exception {
// No regular expressions filtering needed
if (regex==null || regex.isEmpty()) {
return data;
}
List filteredElems = new ArrayList();
// Get each data object to filter
for (Iterator i=data.iterator(); i.hasNext();) {
Object dataElem = i.next();
// If properties of data matches all the regex pattern, add it
if (matches(dataElem, regex)) {
filteredElems.add(dataElem);
}
}
return filteredElems;
}
/**
* Determines how the object is to be matched to the regex map.
* @param data - object to match
* @param regex - regex map
* @return true, if the object matches the regex map, false otherwise
* @throws Exception
*/
protected abstract boolean matches(Object data, Map regex) throws Exception;
}

View File

@ -0,0 +1,76 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
public abstract class ResultTransformFilter implements QueryFilter {
private QueryFilter next;
/**
* Contructs a query filter that transform the format of the query result
* @param next - the query filter to retrieve the results from
*/
protected ResultTransformFilter(QueryFilter next) {
this.next = next;
}
/**
* Transforms the queried results to a collection of different objects
* @param query - the query string
* @return collections of transformed objects
* @throws Exception
*/
public List query(String query) throws Exception {
return transformList(next.query(query));
}
/**
* Transforms the queried results to a collection of different objects
* @param queries - the query map
* @return collections of transformed objects
* @throws Exception
*/
public List query(List queries) throws Exception {
return transformList(next.query(queries));
}
/**
* Transforms a collection to a collection of different objects.
* @param result - the collection to transform
* @return collection of properties objects
*/
protected List transformList(List result) throws Exception {
List props = new ArrayList();
for (Iterator i=result.iterator(); i.hasNext();) {
props.add(transformElement(i.next()));
}
return props;
}
/**
* Transform a result object
* @param obj - the object instance to transform
* @return the transformed object
*/
protected abstract Object transformElement(Object obj) throws Exception;
}

View File

@ -0,0 +1,51 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import java.util.List;
public class StubQueryFilter implements QueryFilter{
private List data;
/**
* Creates a stub query that returns the given collections as the query result
* @param data - the stub query result
*/
public StubQueryFilter(List data) {
this.data = data;
}
/**
* Returns the provided stub data as a stub query result
* @param queryStr - not use
* @return the stub query result
* @throws Exception
*/
public List query(String queryStr) throws Exception {
return data;
}
/**
* Returns the provided stub data as a stub query result
* @param queries - not use
* @return the stub query result
* @throws Exception
*/
public List query(List queries) throws Exception {
return data;
}
}

View File

@ -0,0 +1,68 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
public class WildcardToMsgSelectorTransformFilter extends WildcardTransformFilter{
/**
* Creates a filter that is able to transform a wildcard query to a message selector format
* @param next - next query filter
*/
public WildcardToMsgSelectorTransformFilter(QueryFilter next) {
super(next);
}
/**
* Use to determine if a query string is a wildcard query. A query string is a wildcard query if it is a key-value
* pair with the format <key>=<value> and the value is enclosed in '' and contains '*' and '?'.
* @param query - query string
* @return true, if the query string is a wildcard query, false otherwise
*/
protected boolean isWildcardQuery(String query) {
// If the query is a key=value pair
String key = query;
String val = "";
int pos = key.indexOf("=");
if (pos >= 0) {
val = key.substring(pos + 1);
key = key.substring(0, pos);
}
// If the value contains wildcards and is enclose by '
return val.startsWith("'") && val.endsWith("'") && ((val.indexOf("*") >= 0) || (val.indexOf("?") >= 0));
}
/**
* Transform a wildcard query to message selector format
* @param query - query string to transform
* @return message selector format string
*/
protected String transformWildcardQuery(String query) {
// If the query is a key=value pair
String key = query;
String val = "";
int pos = key.indexOf("=");
if (pos >= 0) {
val = key.substring(pos + 1);
key = key.substring(0, pos);
}
val = val.replaceAll("[?]", "_");
val = val.replaceAll("[*]", "%");
return key + " LIKE " + val;
}
}

View File

@ -0,0 +1,71 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
public class WildcardToRegExTransformFilter extends WildcardTransformFilter {
/**
* Creates a filter that is able to transform a wildcard query to a regular expression query string
* @param next - next query filter
*/
public WildcardToRegExTransformFilter(RegExQueryFilter next) {
super(next);
}
/**
* Use to determine if a query string is a wildcard query. A query string is a wildcard query if it is a key-value
* pair with the format <key>=<value> and the value contains '*' and '?'.
* @param query - query string
* @return true, if the query string is a wildcard query, false otherwise
*/
protected boolean isWildcardQuery(String query) {
// If the query is a key=value pair
String key = query;
String val = "";
int pos = key.indexOf("=");
if (pos >= 0) {
val = key.substring(pos + 1);
key = key.substring(0, pos);
}
// If the value contains wildcards
return ((val.indexOf("*") >= 0) || (val.indexOf("?") >= 0));
}
/**
* Transform a wildcard query to regular expression format
* @param query - query string to transform
* @return regex query string
*/
protected String transformWildcardQuery(String query) {
// Get the key=value pair
String key = query;
String val = "";
int pos = key.indexOf("=");
if (pos >= 0) {
val = key.substring(pos + 1);
key = key.substring(0, pos);
}
val = val.replaceAll("[.]", "\\\\."); // Escape all dot characters. From (.) to (\.)
val = val.replaceAll("[?]", "."); // Match single characters
val = val.replaceAll("[*]", ".*?"); // Match all characters, use reluctant quantifier
val = "(" + val +")"; // Let's group the query for clarity
val = RegExQueryFilter.REGEX_PREFIX + val; // Flag as a regular expression query
return key + "=" + val;
}
}

View File

@ -0,0 +1,73 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.filter;
import java.util.Iterator;
import java.util.ArrayList;
import java.util.List;
public abstract class WildcardTransformFilter extends AbstractQueryFilter {
/**
* Creates a wildcard transform filter that is able to convert a wildcard expression (determined by isWildcardQuery)
* to a another query type (use transformWildcardQuery).
* @param next - the next query filter
*/
protected WildcardTransformFilter(QueryFilter next) {
super(next);
}
/**
* Converts the query list to set of different queries
* @param queries - query list to transform
* @return - result of the query
* @throws Exception
*/
public List query(List queries) throws Exception {
List newQueries = new ArrayList();
for (Iterator i=queries.iterator(); i.hasNext();) {
String queryToken = (String)i.next();
// Transform the wildcard query
if (isWildcardQuery(queryToken)) {
// Transform the value part only
newQueries.add(transformWildcardQuery(queryToken));
// Maintain the query as is
} else {
newQueries.add(queryToken);
}
}
return next.query(newQueries);
}
/**
* Use to determine is a query string is a wildcard query
* @param query - query string
* @return true, if the query string is a wildcard query, false otherwise
*/
protected abstract boolean isWildcardQuery(String query);
/**
* Use to transform a wildcard query string to another query format
* @param query - query string to transform
* @return transformed query
*/
protected abstract String transformWildcardQuery(String query);
}

View File

@ -0,0 +1,243 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.formatter;
import javax.management.ObjectInstance;
import javax.management.ObjectName;
import javax.management.AttributeList;
import javax.management.Attribute;
import javax.jms.Message;
import java.util.Map;
import java.util.Collection;
import java.util.Iterator;
import java.io.PrintStream;
import java.io.OutputStream;
public class CommandShellOutputFormatter implements OutputFormatter {
private OutputStream outputStream;
private PrintStream out;
public CommandShellOutputFormatter(OutputStream out) {
this.outputStream = out;
if (out instanceof PrintStream) {
this.out = (PrintStream)out;
} else {
this.out = new PrintStream(out);
}
}
/**
* Retrieve the output stream being used by the formatter
* @return
*/
public OutputStream getOutputStream() {
return outputStream;
}
/**
* Print an ObjectInstance format of an mbean
* @param mbean - mbean to print
*/
public void printMBean(ObjectInstance mbean) {
printMBean(mbean.getObjectName());
}
/**
* Print an ObjectName format of an mbean
* @param mbean - mbean to print
*/
public void printMBean(ObjectName mbean) {
printMBean(mbean.getKeyPropertyList());
}
/**
* Print an AttributeList format of an mbean
* @param mbean - mbean to print
*/
public void printMBean(AttributeList mbean) {
for (Iterator i=mbean.iterator(); i.hasNext();) {
Attribute attrib = (Attribute)i.next();
if (attrib.getValue() instanceof ObjectName) {
printMBean((ObjectName)attrib.getValue());
} else if (attrib.getValue() instanceof ObjectInstance) {
printMBean((ObjectInstance)attrib.getValue());
} else {
out.println(attrib.getName() + " = " + attrib.getValue().toString());
out.println();
}
}
}
/**
* Print a Map format of an mbean
* @param mbean - mbean to print
*/
public void printMBean(Map mbean) {
for (Iterator i=mbean.keySet().iterator(); i.hasNext();) {
String key = (String)i.next();
String val = mbean.get(key).toString();
out.println(key + " = " + val);
}
out.println();
}
/**
* Print a collection of mbean
* @param mbean - collection of mbeans
*/
public void printMBean(Collection mbean) {
for (Iterator i=mbean.iterator(); i.hasNext();) {
Object obj = i.next();
if (obj instanceof ObjectInstance) {
printMBean((ObjectInstance)obj);
} else if (obj instanceof ObjectName) {
printMBean((ObjectName)obj);
} else if (obj instanceof Map) {
printMBean((Map)obj);
} else if (obj instanceof AttributeList) {
printMBean((AttributeList)obj);
} else if (obj instanceof Collection) {
printMessage((Collection)obj);
} else {
printException(new UnsupportedOperationException("Unknown mbean type: " + obj.getClass().getName()));
}
}
}
/**
* Print a Map format of a JMS message
* @param msg
*/
public void printMessage(Map msg) {
for (Iterator i=msg.keySet().iterator(); i.hasNext();) {
String key = (String)i.next();
String val = msg.get(key).toString();
out.println(key + " = " + val);
}
out.println();
}
/**
* Print a Message format of a JMS message
* @param msg - JMS message to print
*/
public void printMessage(Message msg) {
// TODO
}
/**
* Print a collection of JMS messages
* @param msg - collection of JMS messages
*/
public void printMessage(Collection msg) {
for (Iterator i=msg.iterator(); i.hasNext();) {
Object obj = i.next();
if (obj instanceof Message) {
printMessage((Message)obj);
} else if (obj instanceof Map) {
printMessage((Map)obj);
} else if (obj instanceof Collection) {
printMessage((Collection)obj);
} else {
printException(new UnsupportedOperationException("Unknown message type: " + obj.getClass().getName()));
}
}
}
/**
* Print help messages
* @param helpMsgs - help messages to print
*/
public void printHelp(String[] helpMsgs) {
for (int i=0; i<helpMsgs.length; i++) {
out.println(helpMsgs[i]);
}
out.println();
}
/**
* Print an information message
* @param info - information message to print
*/
public void printInfo(String info) {
out.println("INFO: " + info);
}
/**
* Print an exception message
* @param e - exception to print
*/
public void printException(Exception e) {
out.println("ERROR: " + e);
}
/**
* Print a version information
* @param version - version info to print
*/
public void printVersion(String version) {
out.println("");
out.println("ActiveMQ " + version);
out.println("For help or more information please see: http://www.logicblaze.com");
out.println("");
}
/**
* Print a generic key value mapping
* @param map to print
*/
public void print(Map map) {
for (Iterator i=map.keySet().iterator(); i.hasNext();) {
String key = (String)i.next();
String val = map.get(key).toString();
out.println(key + " = " + val);
}
out.println();
}
/**
* Print a generic array of strings
* @param strings - string array to print
*/
public void print(String[] strings) {
for (int i=0; i<strings.length; i++) {
out.println(strings[i]);
}
out.println();
}
/**
* Print a collection of objects
* @param collection - collection to print
*/
public void print(Collection collection) {
for (Iterator i=collection.iterator(); i.hasNext();) {
out.println(i.next().toString());
}
out.println();
}
/**
* Print a java string
* @param string - string to print
*/
public void print(String string) {
out.println(string);
}
}

View File

@ -0,0 +1,232 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.formatter;
import javax.management.ObjectInstance;
import javax.management.ObjectName;
import javax.management.AttributeList;
import javax.jms.Message;
import java.util.Map;
import java.util.Collection;
import java.io.OutputStream;
public class GlobalWriter {
private static OutputFormatter formatter;
/**
* Creates a singleton global writer
*/
private GlobalWriter() {
}
/**
* Maintains a global output formatter
* @param formatter - the output formatter to use
*/
public static void instantiate(OutputFormatter formatter) {
if (GlobalWriter.formatter == null) {
GlobalWriter.formatter = formatter;
}
}
/**
* Retrieve the output stream being used by the global formatter
* @return
*/
public static OutputStream getOutputStream() {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
return formatter.getOutputStream();
}
/**
* Print an ObjectInstance format of an mbean
* @param mbean - mbean to print
*/
public static void printMBean(ObjectInstance mbean) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.printMBean(mbean);
}
/**
* Print an ObjectName format of an mbean
* @param mbean - mbean to print
*/
public static void printMBean(ObjectName mbean) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.printMBean(mbean);
}
/**
* Print an AttributeList format of an mbean
* @param mbean - mbean to print
*/
public static void printMBean(AttributeList mbean) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.printMBean(mbean);
}
/**
* Print a Map format of an mbean
* @param mbean
*/
public static void printMBean(Map mbean) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.printMBean(mbean);
}
/**
* Print a Collection format of mbeans
* @param mbean - collection of mbeans
*/
public static void printMBean(Collection mbean) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.printMBean(mbean);
}
/**
* Print a Map format of a JMS message
* @param msg
*/
public static void printMessage(Map msg) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.printMessage(msg);
}
/**
* Print a Message format of a JMS message
* @param msg - JMS message to print
*/
public static void printMessage(Message msg) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.printMessage(msg);
}
/**
* Print a collection of JMS messages
* @param msg - collection of JMS messages
*/
public static void printMessage(Collection msg) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.printMessage(msg);
}
/**
* Print help messages
* @param helpMsgs - help messages to print
*/
public static void printHelp(String[] helpMsgs) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.printHelp(helpMsgs);
}
/**
* Print an information message
* @param info - information message to print
*/
public static void printInfo(String info) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.printInfo(info);
}
/**
* Print an exception message
* @param e - exception to print
*/
public static void printException(Exception e) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.printException(e);
}
/**
* Print a version information
* @param version - version info to print
*/
public static void printVersion(String version) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.printVersion(version);
}
/**
* Print a generic key value mapping
* @param map to print
*/
public static void print(Map map) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.print(map);
}
/**
* Print a generic array of strings
* @param strings - string array to print
*/
public static void print(String[] strings) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.print(strings);
}
/**
* Print a collection of objects
* @param collection - collection to print
*/
public static void print(Collection collection) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.print(collection);
}
/**
* Print a java string
* @param string - string to print
*/
public static void print(String string) {
if (formatter == null) {
throw new IllegalStateException("No OutputFormatter specified. Use GlobalWriter.instantiate(OutputFormatter).");
}
formatter.print(string);
}
}

View File

@ -0,0 +1,130 @@
/**
*
* Copyright 2005-2006 The Apache Software Foundation
*
* Licensed 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.activemq.broker.console.formatter;
import javax.management.ObjectInstance;
import javax.management.ObjectName;
import javax.management.AttributeList;
import javax.jms.Message;
import java.util.Collection;
import java.util.Map;
import java.io.OutputStream;
public interface OutputFormatter {
/**
* Retrieve the output stream being used by the formatter
* @return
*/
public OutputStream getOutputStream();
/**
* Print an ObjectInstance format of an mbean
* @param mbean - mbean to print
*/
public void printMBean(ObjectInstance mbean);
/**
* Print an ObjectName format of an mbean
* @param mbean - mbean to print
*/
public void printMBean(ObjectName mbean);
/**
* Print an AttributeList format of an mbean
* @param mbean - mbean to print
*/
public void printMBean(AttributeList mbean);
/**
* Print a Map format of an mbean
* @param mbean - mbean to print
*/
public void printMBean(Map mbean);
/**
* Print a Collection format of mbeans
* @param mbean - collection of mbeans
*/
public void printMBean(Collection mbean);
/**
* Print a Map format of a JMS message
* @param msg
*/
public void printMessage(Map msg);
/**
* Print a Message format of a JMS message
* @param msg - JMS message to print
*/
public void printMessage(Message msg);
/**
* Print a Collection format of JMS messages
* @param msg - collection of JMS messages
*/
public void printMessage(Collection msg);
/**
* Print help messages
* @param helpMsgs - help messages to print
*/
public void printHelp(String[] helpMsgs);
/**
* Print an information message
* @param info - information message to print
*/
public void printInfo(String info);
/**
* Print an exception message
* @param e - exception to print
*/
public void printException(Exception e);
/**
* Print a version information
* @param version - version info to print
*/
public void printVersion(String version);
/**
* Print a generic key value mapping
* @param map to print
*/
public void print(Map map);
/**
* Print a generic array of strings
* @param strings - string array to print
*/
public void print(String[] strings);
/**
* Print a collection of objects
* @param collection - collection to print
*/
public void print(Collection collection);
/**
* Print a java string
* @param string - string to print
*/
public void print(String string);
}

View File

@ -0,0 +1,142 @@
#!/bin/sh
# ActiveMQ shell script
#
# This script is heavily based on the Ant script
#
# Copyright 2005-2006 The Apache Software Foundation
#
# Licensed 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.
# load system-wide activemq configuration
if [ -f "/etc/activemq.conf" ] ; then
. /etc/activemq.conf
fi
# provide default values for people who don't use RPMs
if [ -z "$usejikes" ] ; then
usejikes=false;
fi
# load user activemq configuration
if [ -f "$HOME/.activemqrc" ] ; then
. "$HOME/.activemqrc"
fi
# OS specific support. $var _must_ be set to either true or false.
cygwin=false;
darwin=false;
case "`uname`" in
CYGWIN*) cygwin=true ;;
Darwin*) darwin=true
if [ -z "$JAVA_HOME" ] ; then
JAVA_HOME=/System/Library/Frameworks/JavaVM.framework/Home
fi
;;
esac
if [ -z "$ACTIVEMQ_HOME" ] ; then
# try to find ACTIVEMQ
if [ -d /opt/activemq ] ; then
ACTIVEMQ_HOME=/opt/activemq
fi
if [ -d "${HOME}/opt/activemq" ] ; then
ACTIVEMQ_HOME="${HOME}/opt/activemq"
fi
## resolve links - $0 may be a link to activemq's home
PRG="$0"
progname=`basename "$0"`
saveddir=`pwd`
# need this for relative symlinks
dirname_prg=`dirname "$PRG"`
cd "$dirname_prg"
while [ -h "$PRG" ] ; do
ls=`ls -ld "$PRG"`
link=`expr "$ls" : '.*-> \(.*\)$'`
if expr "$link" : '.*/.*' > /dev/null; then
PRG="$link"
else
PRG=`dirname "$PRG"`"/$link"
fi
done
ACTIVEMQ_HOME=`dirname "$PRG"`/..
cd "$saveddir"
# make it fully qualified
ACTIVEMQ_HOME=`cd "$ACTIVEMQ_HOME" && pwd`
fi
# For Cygwin, ensure paths are in UNIX format before anything is touched
if $cygwin ; then
[ -n "$ACTIVEMQ_HOME" ] &&
ACTIVEMQ_HOME=`cygpath --unix "$ACTIVEMQ_HOME"`
[ -n "$JAVA_HOME" ] &&
JAVA_HOME=`cygpath --unix "$JAVA_HOME"`
[ -n "$CLASSPATH" ] &&
CLASSPATH=`cygpath --path --unix "$CLASSPATH"`
fi
if [ -z "$JAVACMD" ] ; then
if [ -n "$JAVA_HOME" ] ; then
if [ -x "$JAVA_HOME/jre/sh/java" ] ; then
# IBM's JDK on AIX uses strange locations for the executables
JAVACMD="$JAVA_HOME/jre/sh/java"
else
JAVACMD="$JAVA_HOME/bin/java"
fi
else
JAVACMD=`which java 2> /dev/null `
if [ -z "$JAVACMD" ] ; then
JAVACMD=java
fi
fi
fi
if [ ! -x "$JAVACMD" ] ; then
echo "Error: JAVA_HOME is not defined correctly."
echo " We cannot execute $JAVACMD"
exit 1
fi
# For Cygwin, switch paths to Windows format before running java
if $cygwin; then
ACTIVEMQ_HOME=`cygpath --windows "$ACTIVEMQ_HOME"`
JAVA_HOME=`cygpath --windows "$JAVA_HOME"`
CLASSPATH=`cygpath --path --windows "$CLASSPATH"`
CYGHOME=`cygpath --windows "$HOME"`
fi
if [ -z "$ACTIVEMQ_OPTS" ] ; then
ACTIVEMQ_OPTS="-Xmx512M -Dderby.system.home=../data -Dderby.storage.fileSyncTransactionLog=true"
fi
# Uncomment to enable YourKit profiling
#ACTIVEMQ_DEBUG_OPTS="-Xrunyjpagent"
# Uncomment to enable remote debugging
#ACTIVEMQ_DEBUG_OPTS="-Xdebug -Xnoagent -Djava.compiler=NONE -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5005"
ACTIVEMQ_TASK="browse"
if [ -n "$CYGHOME" ]; then
exec "$JAVACMD" $ACTIVEMQ_DEBUG_OPTS $ACTIVEMQ_OPTS -classpath "" -Dactivemq.home="${ACTIVEMQ_HOME}" -Dcygwin.user.home="$CYGHOME" -jar ${ACTIVEMQ_HOME}/bin/run.jar $ACTIVEMQ_TASK $@
else
exec "$JAVACMD" $ACTIVEMQ_DEBUG_OPTS $ACTIVEMQ_OPTS -classpath "" -Dactivemq.home="${ACTIVEMQ_HOME}" -jar ${ACTIVEMQ_HOME}/bin/run.jar $ACTIVEMQ_TASK $@
fi

View File

@ -0,0 +1,115 @@
@echo off
REM ActiveMQ shell script
REM
REM This script is heavily based on the Ant script
REM
REM Copyright 2005-2006 The Apache Software Foundation
REM
REM Licensed under the Apache License, Version 2.0 (the "License");
REM you may not use this file except in compliance with the License.
REM You may obtain a copy of the License at
REM
REM http://www.apache.org/licenses/LICENSE-2.0
REM
REM Unless required by applicable law or agreed to in writing, software
REM distributed under the License is distributed on an "AS IS" BASIS,
REM WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
REM See the License for the specific language governing permissions and
REM limitations under the License.
if exist "%HOME%\activemqrc_pre.bat" call "%HOME%\activemqrc_pre.bat"
if "%OS%"=="Windows_NT" @setlocal
rem %~dp0 is expanded pathname of the current script under NT
set DEFAULT_ACTIVEMQ_HOME=%~dp0..
if "%ACTIVEMQ_HOME%"=="" set ACTIVEMQ_HOME=%DEFAULT_ACTIVEMQ_HOME%
set DEFAULT_ACTIVEMQ_HOME=
rem Slurp the command line arguments. This loop allows for an unlimited number
rem of arguments (up to the command line limit, anyway).
set ACTIVEMQ_CMD_LINE_ARGS=%1
if ""%1""=="""" goto doneStart
shift
:setupArgs
if ""%1""=="""" goto doneStart
set ACTIVEMQ_CMD_LINE_ARGS=%ACTIVEMQ_CMD_LINE_ARGS% %1
shift
goto setupArgs
rem This label provides a place for the argument list loop to break out
rem and for NT handling to skip to.
:doneStart
rem find ACTIVEMQ_HOME if it does not exist due to either an invalid value passed
rem by the user or the %0 problem on Windows 9x
if exist "%ACTIVEMQ_HOME%\README.txt" goto checkJava
rem check for activemq in Program Files on system drive
if not exist "%SystemDrive%\Program Files\activemq" goto checkSystemDrive
set ACTIVEMQ_HOME=%SystemDrive%\Program Files\activemq
goto checkJava
:checkSystemDrive
rem check for activemq in root directory of system drive
if not exist %SystemDrive%\activemq\README.txt goto checkCDrive
set ACTIVEMQ_HOME=%SystemDrive%\activemq
goto checkJava
:checkCDrive
rem check for activemq in C:\activemq for Win9X users
if not exist C:\activemq\README.txt goto noAntHome
set ACTIVEMQ_HOME=C:\activemq
goto checkJava
:noAntHome
echo ACTIVEMQ_HOME is set incorrectly or activemq could not be located. Please set ACTIVEMQ_HOME.
goto end
:checkJava
set _JAVACMD=%JAVACMD%
set LOCALCLASSPATH=%CLASSPATH%
set JAVA_EXT_DIRS=%JAVA_HOME%\lib\ext;%ACTIVEMQ_HOME%;%ACTIVEMQ_HOME%\lib;%ACTIVEMQ_HOME%\lib\optional
if "%JAVA_HOME%" == "" goto noJavaHome
if not exist "%JAVA_HOME%\bin\java.exe" goto noJavaHome
if "%_JAVACMD%" == "" set _JAVACMD=%JAVA_HOME%\bin\java.exe
goto runAnt
:noJavaHome
if "%_JAVACMD%" == "" set _JAVACMD=java.exe
echo.
echo Warning: JAVA_HOME environment variable is not set.
echo.
:runAnt
if "%ACTIVEMQ_OPTS%" == "" set ACTIVEMQ_OPTS=-Xmx512M -Dderby.system.home="..\data" -Dderby.storage.fileSyncTransactionLog=true
REM Uncomment to enable YourKit profiling
REM SET ACTIVEMQ_DEBUG_OPTS="-Xrunyjpagent"
REM Uncomment to enable remote debugging
REM SET ACTIVEMQ_DEBUG_OPTS=-Xdebug -Xnoagent -Djava.compiler=NONE -Xrunjdwp:transport=dt_socket,server=y,suspend=n,address=5005
set LOCALCLASSPATH=%ACTIVEMQ_HOME%\conf;%LOCALCLASSPATH%
set ACTIVEMQ_TASK="browse"
"%_JAVACMD%" %ACTIVEMQ_DEBUG_OPTS% %ACTIVEMQ_OPTS% -Djava.ext.dirs="%JAVA_EXT_DIRS%" -classpath "%LOCALCLASSPATH%" -jar %ACTIVEMQ_HOME%/bin/run.jar %ACTIVEMQ_TASK% %ACTIVEMQ_CMD_LINE_ARGS%
goto end
:end
set LOCALCLASSPATH=
set _JAVACMD=
set ACTIVEMQ_CMD_LINE_ARGS=
if "%OS%"=="Windows_NT" @endlocal
:mainEnd
if exist "%HOME%\activemqrc_post.bat" call "%HOME%\activemqrc_post.bat"

View File

@ -142,7 +142,7 @@ fi
# Select all components that belongs to the specified broker except advisory topics
# and display the specified attributes
QUERY_PARAM="--objname Type=*,BrokerName=${BROKER_NAME} -xQTopic=ActiveMQ.Advisory.* --view EnqueueCount,DequeueCount,TotalEnqueueCount,TotalDequeueCount,Messages,TotalMessages,ConsumerCount,TotalConsumerCount,DispatchQueueSize"
QUERY_PARAM="--objname Type=*,BrokerName=${BROKER_NAME} -xQTopic=ActiveMQ.Advisory.* --view Type,BrokerName,Destination,ConnectorName,EnqueueCount,DequeueCount,TotalEnqueueCount,TotalDequeueCount,Messages,TotalMessages,ConsumerCount,TotalConsumerCount,DispatchQueueSize"
if [ -n "$CYGHOME" ]; then
exec "$JAVACMD" $ACTIVEMQ_DEBUG_OPTS $ACTIVEMQ_OPTS -classpath "" -Dactivemq.home="${ACTIVEMQ_HOME}" -Dcygwin.user.home="$CYGHOME" -jar ${ACTIVEMQ_HOME}/bin/run.jar $ACTIVEMQ_TASK $QUERY_PARAM

View File

@ -105,7 +105,7 @@ if "%BROKER_NAME%" == "" set BROKER_NAME=*
rem Select all components that belongs to the specified broker except advisory topics
rem and display the specified attributes
set QUERY_PARAM=--objname "Type=*,BrokerName=%BROKER_NAME%" "-xQTopic=ActiveMQ.Advisory.*" --view "EnqueueCount,DequeueCount,TotalEnqueueCount,TotalDequeueCount,Messages,TotalMessages,ConsumerCount,TotalConsumerCount,DispatchQueueSize"
set QUERY_PARAM=--objname "Type=*,BrokerName=%BROKER_NAME%" "-xQTopic=ActiveMQ.Advisory.*" --view "Type,BrokerName,Destination,ConnectorName,EnqueueCount,DequeueCount,TotalEnqueueCount,TotalDequeueCount,Messages,TotalMessages,ConsumerCount,TotalConsumerCount,DispatchQueueSize"
"%_JAVACMD%" %ACTIVEMQ_DEBUG_OPTS% %ACTIVEMQ_OPTS% -Djava.ext.dirs="%JAVA_EXT_DIRS%" -classpath "%LOCALCLASSPATH%" -jar %ACTIVEMQ_HOME%/bin/run.jar %ACTIVEMQ_TASK% %QUERY_PARAM%