This commit is contained in:
Justin Bertram 2018-01-31 15:13:31 -06:00
commit 7947bcf5d5
6 changed files with 455 additions and 174 deletions

View File

@ -42,7 +42,6 @@ import org.apache.activemq.artemis.core.transaction.Transaction;
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
import org.apache.activemq.artemis.spi.core.protocol.SessionCallback;
import org.apache.activemq.artemis.utils.critical.CriticalComponent;
import org.jboss.logging.Logger;
/**
* plugin to log various events within the broker, configured with the following booleans
@ -57,8 +56,6 @@ import org.jboss.logging.Logger;
public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serializable {
private static final Logger logger = Logger.getLogger(LoggingActiveMQServerPlugin.class);
private static final long serialVersionUID = 1L;
public static final String LOG_ALL_EVENTS = "LOG_ALL_EVENTS";
@ -123,7 +120,7 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
logSendingEvents = Boolean.parseBoolean(properties.getOrDefault(LOG_SENDING_EVENTS, "false"));
logInternalEvents = Boolean.parseBoolean(properties.getOrDefault(LOG_INTERNAL_EVENTS, "false"));
if (logger.isDebugEnabled()) {
if (LoggingActiveMQServerPluginLogger.LOGGER.isDebugEnabled()) {
dumpConfiguration();
}
@ -137,8 +134,8 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void afterCreateConnection(RemotingConnection connection) throws ActiveMQException {
if (logger.isInfoEnabled() && (logAll || logConnectionEvents)) {
logger.infof("created connection: %s", connection);
if (logAll || logConnectionEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.afterCreateConnection(connection);
}
}
@ -150,8 +147,8 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void afterDestroyConnection(RemotingConnection connection) throws ActiveMQException {
if (logger.isInfoEnabled() && (logAll || logConnectionEvents)) {
logger.infof("destroyed connection: %s", connection);
if (logAll || logConnectionEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.afterDestroyConnection(connection);
}
}
@ -188,11 +185,10 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
OperationContext context,
Map<SimpleString, RoutingType> prefixes) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logSessionEvents)) {
logger.debugf("beforeCreateSession called with name: %s , username: %s, minLargeMessageSize: %s , connection: %s"
+ ", autoCommitSends: %s, autoCommitAcks: %s, preAcknowledge: %s, xa: %s, publicAddress: %s"
+ ", , autoCreateQueues: %s, context: %s ", name, username, minLargeMessageSize, connection,
autoCommitSends, autoCommitAcks, preAcknowledge, xa, publicAddress, autoCreateQueues, context);
if (logAll || logSessionEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.beforeCreateSession(name, username, minLargeMessageSize, connection,
autoCommitSends, autoCommitAcks, preAcknowledge,
xa, publicAddress, context);
}
}
@ -205,10 +201,9 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void afterCreateSession(ServerSession session) throws ActiveMQException {
if (logger.isInfoEnabled() && (logAll || logSessionEvents)) {
logger.infof("created session name: %s, session connectionID: %s",
(session == null ? UNAVAILABLE : session.getName()),
(session == null ? UNAVAILABLE : session.getConnectionID()));
if (logAll || logSessionEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.afterCreateSession((session == null ? UNAVAILABLE : session.getName()),
(session == null ? UNAVAILABLE : session.getConnectionID()));
}
}
@ -222,9 +217,9 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void beforeCloseSession(ServerSession session, boolean failed) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logSessionEvents)) {
logger.debugf("beforeCloseSession called with session name : %s, session: %s, failed: %s",
(session == null ? UNAVAILABLE : session.getName()), session, failed);
if (logAll || logSessionEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.beforeCloseSession((session == null ? UNAVAILABLE : session.getName()),
session, failed);
}
}
@ -237,9 +232,9 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void afterCloseSession(ServerSession session, boolean failed) throws ActiveMQException {
if (logger.isInfoEnabled() && (logAll || logSessionEvents)) {
logger.infof("closed session with session name: %s, failed: %s",
(session == null ? UNAVAILABLE : session.getName()), failed);
if (logAll || logSessionEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.afterCloseSession((session == null ? UNAVAILABLE : session.getName()),
failed);
}
}
@ -253,10 +248,9 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void beforeSessionMetadataAdded(ServerSession session, String key, String data) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logSessionEvents)) {
logger.debugf("beforeSessionMetadataAdded called with session name: %s , session: %s, key: %s, data: %s",
(session == null ? UNAVAILABLE : session.getName()), session, key, data);
if (logAll || logSessionEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.beforeSessionMetadataAdded((session == null ? UNAVAILABLE : session.getName()),
session, key, data);
}
}
@ -270,14 +264,15 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void afterSessionMetadataAdded(ServerSession session, String key, String data) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logSessionEvents)) {
logger.debugf("added session metadata for session name : %s, session: %s, key: %s, data: %s",
(session == null ? UNAVAILABLE : session.getName()), session, key, data);
if (logAll || logSessionEvents) {
} else if (logger.isInfoEnabled() && (logAll || logSessionEvents)) {
logger.infof("added session metadata for session name : %s, key: %s, data: %s",
(session == null ? UNAVAILABLE : session.getName()), key, data);
//Details - debug level
LoggingActiveMQServerPluginLogger.LOGGER.afterSessionMetadataAddedDetails((session == null ? UNAVAILABLE : session.getName()),
session, key, data);
// info level log
LoggingActiveMQServerPluginLogger.LOGGER.afterSessionMetadataAdded((session == null ? UNAVAILABLE : session.getName()),
key, data);
}
}
@ -298,10 +293,11 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
boolean browseOnly,
boolean supportLargeMessage) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logConsumerEvents)) {
logger.debugf("beforeCreateConsumer called with ConsumerID: %s, QueueBinding: %s, filterString: %s," +
" browseOnly: %s, supportLargeMessage: %s", consumerID, queueBinding, filterString, browseOnly,
supportLargeMessage);
if (logAll || logConsumerEvents) {
if (LoggingActiveMQServerPluginLogger.LOGGER.isInfoEnabled()) {
LoggingActiveMQServerPluginLogger.LOGGER.beforeCreateConsumer(Long.toString(consumerID), queueBinding,
filterString, browseOnly, supportLargeMessage);
}
}
}
@ -315,10 +311,11 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
@Override
public void afterCreateConsumer(ServerConsumer consumer) throws ActiveMQException {
if (logger.isInfoEnabled() && (logAll || logConsumerEvents)) {
logger.infof("created consumer with ID: %s, with session name: %s",
(consumer == null ? UNAVAILABLE : consumer.getID()),
(consumer == null ? UNAVAILABLE : consumer.getSessionID()));
if (logAll || logConsumerEvents) {
if (LoggingActiveMQServerPluginLogger.LOGGER.isInfoEnabled()) {
LoggingActiveMQServerPluginLogger.LOGGER.afterCreateConsumer((consumer == null ? UNAVAILABLE : Long.toString(consumer.getID())),
(consumer == null ? UNAVAILABLE : consumer.getSessionID()));
}
}
}
@ -333,9 +330,9 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
@Override
public void beforeCloseConsumer(ServerConsumer consumer, boolean failed) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logConsumerEvents)) {
logger.debugf("beforeCloseConsumer called with consumer: %s, consumer sessionID: %s, failed: %s",
consumer, (consumer == null ? UNAVAILABLE : consumer.getSessionID()), failed);
if (logAll || logConsumerEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.beforeCloseConsumer(consumer, (consumer == null ? UNAVAILABLE : consumer.getSessionID()),
failed);
}
}
@ -349,10 +346,11 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
@Override
public void afterCloseConsumer(ServerConsumer consumer, boolean failed) throws ActiveMQException {
if (logger.isInfoEnabled() && (logAll || logConsumerEvents)) {
logger.infof("closed consumer ID: %s, with consumer Session: %s, failed: %s",
(consumer == null ? UNAVAILABLE : consumer.getID()),
(consumer == null ? UNAVAILABLE : consumer.getSessionID()), failed);
if (logAll || logConsumerEvents) {
if (LoggingActiveMQServerPluginLogger.LOGGER.isInfoEnabled()) {
LoggingActiveMQServerPluginLogger.LOGGER.afterCloseConsumer((consumer == null ? UNAVAILABLE : Long.toString(consumer.getID())),
(consumer == null ? UNAVAILABLE : consumer.getSessionID()), failed);
}
}
}
@ -365,8 +363,8 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void beforeCreateQueue(QueueConfig queueConfig) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logInternalEvents)) {
logger.debugf("beforeCreateQueue called with queueConfig: %s", queueConfig);
if (logAll || logInternalEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.beforeCreateQueue(queueConfig);
}
}
@ -379,8 +377,8 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void afterCreateQueue(Queue queue) throws ActiveMQException {
if (logger.isInfoEnabled() && (logAll || logInternalEvents)) {
logger.infof("created queue: %s", queue);
if (logAll || logInternalEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.afterCreateQueue(queue);
}
}
@ -400,11 +398,9 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
boolean checkConsumerCount,
boolean removeConsumers,
boolean autoDeleteAddress) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logInternalEvents)) {
logger.debugf("beforeDestroyQueue called with queueName: %s, session: %s, checkConsumerCount: %s," +
" removeConsumers: %s, autoDeleteAddress: %s", queueName, session, checkConsumerCount,
removeConsumers, autoDeleteAddress);
if (logAll || logInternalEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.beforeDestroyQueue(queueName, session, checkConsumerCount, removeConsumers,
autoDeleteAddress);
}
}
@ -428,11 +424,9 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
boolean removeConsumers,
boolean autoDeleteAddress) throws ActiveMQException {
if (logger.isInfoEnabled() && (logAll || logInternalEvents)) {
logger.infof("destroyed queue: %s, with args address: %s, session: %s, checkConsumerCount: %s," +
" removeConsumers: %s, autoDeleteAddress: %s", queue, address, session, checkConsumerCount,
removeConsumers, autoDeleteAddress);
if (logAll || logInternalEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.afterDestroyQueue(queue, address, session, checkConsumerCount,
removeConsumers, autoDeleteAddress);
}
}
@ -454,9 +448,8 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
boolean direct,
boolean noAutoCreateQueue) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logSendingEvents)) {
logger.debugf("beforeSend called with message: %s, tx: %s, session: %s, direct: %s, noAutoCreateQueue: %s",
message, tx, session, direct, noAutoCreateQueue);
if (logAll || logSendingEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.beforeSend(message, tx, session, direct, noAutoCreateQueue);
}
}
@ -479,17 +472,22 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
boolean noAutoCreateQueue,
RoutingStatus result) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logSendingEvents)) {
logger.debugf("sent message: %s, session name : %s, session connectionID: %s, with tx: %s, session: %s," +
" direct: %s, noAutoCreateQueue: %s, result: %s", message,
(session == null ? UNAVAILABLE : session.getName()),
(session == null ? UNAVAILABLE : session.getConnectionID()),
tx, session, direct, noAutoCreateQueue, result);
} else if (logger.isInfoEnabled() && (logAll || logSendingEvents)) {
logger.infof("sent message with ID: %s, session name: %s, session connectionID: %s, result: %s",
(message == null ? UNAVAILABLE : message.getMessageID()),
(session == null ? UNAVAILABLE : session.getName()),
(session == null ? UNAVAILABLE : session.getConnectionID()), result);
if (logAll || logSendingEvents) {
if (LoggingActiveMQServerPluginLogger.LOGGER.isDebugEnabled()) {
//details - debug level
LoggingActiveMQServerPluginLogger.LOGGER.afterSendDetails((message == null ? UNAVAILABLE : Long.toString(message.getMessageID())),
message, (session == null ? UNAVAILABLE : session.getName()),
tx, session, direct, noAutoCreateQueue);
}
if (LoggingActiveMQServerPluginLogger.LOGGER.isInfoEnabled()) {
//info level log
LoggingActiveMQServerPluginLogger.LOGGER.afterSend((message == null ? UNAVAILABLE : Long.toString(message.getMessageID())),
(session == null ? UNAVAILABLE : session.getName()),
(session == null ? UNAVAILABLE : session.getConnectionID().toString()),
result);
}
}
}
@ -507,9 +505,8 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
RoutingContext context,
boolean direct,
boolean rejectDuplicates) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logSendingEvents)) {
logger.debugf("beforeMessageRoute called with message: %s, context: %s, direct: %s, rejectDuplicates: %s",
message, context, direct, rejectDuplicates);
if (logAll || logSendingEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.beforeMessageRoute(message, context, direct, rejectDuplicates);
}
}
@ -529,13 +526,16 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
boolean direct,
boolean rejectDuplicates,
RoutingStatus result) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logSendingEvents)) {
logger.debugf("routed message: %s, with context: %s, direct: %s, rejectDuplicates: %s, result: %s",
message, context, direct, rejectDuplicates, result);
if (logAll || logSendingEvents) {
} else if (logger.isInfoEnabled() && (logAll || logSendingEvents)) {
logger.infof("routed message with ID: %s, result: %s",
(message == null ? UNAVAILABLE : message.getMessageID()), result);
//details - debug level logging
LoggingActiveMQServerPluginLogger.LOGGER.afterMessageRouteDetails(message, context, direct, rejectDuplicates);
if (LoggingActiveMQServerPluginLogger.LOGGER.isInfoEnabled()) {
//info level log
LoggingActiveMQServerPluginLogger.LOGGER.afterMessageRoute((message == null ? UNAVAILABLE : Long.toString(message.getMessageID())),
result);
}
}
}
@ -549,8 +549,9 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void beforeDeliver(ServerConsumer consumer, MessageReference reference) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logDeliveringEvents)) {
logger.debugf("beforeDeliver called with consumer: %s, reference: %s", consumer, reference);
if (logAll || logDeliveringEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.beforeDeliver(consumer, reference);
}
}
@ -563,26 +564,32 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void afterDeliver(ServerConsumer consumer, MessageReference reference) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logDeliveringEvents)) {
if (logAll || logDeliveringEvents) {
Message message = (reference == null ? null : reference.getMessage());
if (consumer != null) {
logger.debugf("delivered message with message ID: %s to consumer on address: %s, queue: %s, consumer sessionID: %s,"
+ " consumerID: %s, full message: %s, full consumer: %s", (message == null ? UNAVAILABLE : message.getMessageID()),
consumer.getQueueAddress(), consumer.getQueueName(), consumer.getSessionID(), consumer.getID(), reference, consumer);
} else {
logger.debugf("delivered message with message ID: %s, consumer info UNAVAILABLE", (message == null ? UNAVAILABLE : message.getMessageID()));
if (consumer == null) {
// log at info level and exit
if (LoggingActiveMQServerPluginLogger.LOGGER.isInfoEnabled()) {
LoggingActiveMQServerPluginLogger.LOGGER.afterDeliverNoConsumer((message == null ? UNAVAILABLE : Long.toString(message.getMessageID())));
}
return;
}
} else if (logger.isInfoEnabled() && (logAll || logDeliveringEvents)) {
Message message = (reference == null ? null : reference.getMessage());
if (LoggingActiveMQServerPluginLogger.LOGGER.isDebugEnabled()) {
//details --- debug level
LoggingActiveMQServerPluginLogger.LOGGER.afterDeliverDetails((message == null ? UNAVAILABLE : Long.toString(message.getMessageID())),
consumer.getQueueAddress(), consumer.getQueueName(),
consumer.getSessionID(), consumer.getID(),
reference, consumer);
}
if (consumer != null) {
logger.infof("delivered message with message ID: %s, to consumer on address: %s, queue: %s, consumer sessionID: %s, consumerID: %s",
(message == null ? UNAVAILABLE : message.getMessageID()), consumer.getQueueAddress(), consumer.getQueueName(),
consumer.getSessionID(), consumer.getID());
} else {
logger.infof("delivered message with message ID: %s, consumer info UNAVAILABLE", (message == null ? UNAVAILABLE : message.getMessageID()));
if (LoggingActiveMQServerPluginLogger.LOGGER.isInfoEnabled()) {
//info level log
LoggingActiveMQServerPluginLogger.LOGGER.afterDeliver((message == null ? UNAVAILABLE : Long.toString(message.getMessageID())),
consumer.getQueueAddress(),
consumer.getQueueName(),
consumer.getSessionID(), consumer.getID());
}
}
@ -597,8 +604,8 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void messageExpired(MessageReference message, SimpleString messageExpiryAddress) throws ActiveMQException {
if (logger.isInfoEnabled() && (logAll || logInternalEvents)) {
logger.infof("expired message: %s, messageExpiryAddress: %s", message, messageExpiryAddress);
if (logAll || logInternalEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.messageExpired(message, messageExpiryAddress);
}
}
@ -611,15 +618,21 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void messageAcknowledged(MessageReference ref, AckReason reason) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logDeliveringEvents)) {
logger.debugf("acknowledged message: %s, with ackReason: %s", ref, reason);
} else if (logger.isInfoEnabled() && (logAll || logDeliveringEvents)) {
Message message = (ref == null ? null : ref.getMessage());
Queue queue = (ref == null ? null : ref.getQueue());
if (logAll || logDeliveringEvents) {
logger.infof("acknowledged message ID: %s, with messageRef consumerID: %s, messageRef QueueName: %s, with ackReason: %s",
(message == null ? UNAVAILABLE : message.getMessageID()), (ref == null ? UNAVAILABLE : ref.getConsumerId()),
(queue == null ? UNAVAILABLE : queue.getName()), reason);
//details - debug logging
LoggingActiveMQServerPluginLogger.LOGGER.messageAcknowledgedDetails(ref, reason);
if (LoggingActiveMQServerPluginLogger.LOGGER.isInfoEnabled()) {
Message message = (ref == null ? null : ref.getMessage());
Queue queue = (ref == null ? null : ref.getQueue());
// info level logging
LoggingActiveMQServerPluginLogger.LOGGER.messageAcknowledged((message == null ? UNAVAILABLE : Long.toString(message.getMessageID())),
(ref == null ? UNAVAILABLE : Long.toString(ref.getConsumerId())),
(queue == null ? UNAVAILABLE : queue.getName().toString()),
reason);
}
}
}
@ -631,8 +644,8 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void beforeDeployBridge(BridgeConfiguration config) throws ActiveMQException {
if (logger.isDebugEnabled() && (logAll || logInternalEvents)) {
logger.debugf("beforeDeployBridge called with bridgeConfiguration: %s", config);
if (logAll || logInternalEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.beforeDeployBridge(config);
}
}
@ -644,8 +657,8 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void afterDeployBridge(Bridge bridge) throws ActiveMQException {
if (logger.isInfoEnabled() && (logAll || logInternalEvents)) {
logger.infof("deployed bridge: %s", bridge);
if (logAll || logInternalEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.afterDeployBridge(bridge);
}
}
@ -658,8 +671,8 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
*/
@Override
public void criticalFailure(CriticalComponent components) throws ActiveMQException {
if (logger.isInfoEnabled() && (logAll || logInternalEvents)) {
logger.infof("criticalFailure called with criticalComponent: %s", components);
if (logAll || logInternalEvents) {
LoggingActiveMQServerPluginLogger.LOGGER.criticalFailure(components);
}
}
@ -667,14 +680,14 @@ public class LoggingActiveMQServerPlugin implements ActiveMQServerPlugin, Serial
* dump the configuration of the logging Plugin
*/
private void dumpConfiguration() {
if (logger.isDebugEnabled()) {
logger.debug("LoggingPlugin logAll=" + logAll);
logger.debug("LoggingPlugin logConnectionEvents=" + logConnectionEvents);
logger.debug("LoggingPlugin logSessionEvents=" + logSessionEvents);
logger.debug("LoggingPlugin logConsumerEvents=" + logConsumerEvents);
logger.debug("LoggingPlugin logSendingEvents=" + logSendingEvents);
logger.debug("LoggingPlugin logDeliveringEvents=" + logDeliveringEvents);
logger.debug("LoggingPlugin logInternalEvents=" + logInternalEvents);
if (LoggingActiveMQServerPluginLogger.LOGGER.isDebugEnabled()) {
LoggingActiveMQServerPluginLogger.LOGGER.debug("LoggingPlugin logAll=" + logAll);
LoggingActiveMQServerPluginLogger.LOGGER.debug("LoggingPlugin logConnectionEvents=" + logConnectionEvents);
LoggingActiveMQServerPluginLogger.LOGGER.debug("LoggingPlugin logSessionEvents=" + logSessionEvents);
LoggingActiveMQServerPluginLogger.LOGGER.debug("LoggingPlugin logConsumerEvents=" + logConsumerEvents);
LoggingActiveMQServerPluginLogger.LOGGER.debug("LoggingPlugin logSendingEvents=" + logSendingEvents);
LoggingActiveMQServerPluginLogger.LOGGER.debug("LoggingPlugin logDeliveringEvents=" + logDeliveringEvents);
LoggingActiveMQServerPluginLogger.LOGGER.debug("LoggingPlugin logInternalEvents=" + logInternalEvents);
}
}

View File

@ -0,0 +1,261 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.activemq.artemis.core.server.plugin.impl;
import org.apache.activemq.artemis.api.core.SimpleString;
import org.apache.activemq.artemis.core.config.BridgeConfiguration;
import org.apache.activemq.artemis.core.persistence.OperationContext;
import org.apache.activemq.artemis.core.postoffice.QueueBinding;
import org.apache.activemq.artemis.core.postoffice.RoutingStatus;
import org.apache.activemq.artemis.core.security.SecurityAuth;
import org.apache.activemq.artemis.core.server.MessageReference;
import org.apache.activemq.artemis.core.server.Queue;
import org.apache.activemq.artemis.core.server.QueueConfig;
import org.apache.activemq.artemis.core.server.RoutingContext;
import org.apache.activemq.artemis.core.server.ServerConsumer;
import org.apache.activemq.artemis.core.server.ServerSession;
import org.apache.activemq.artemis.core.server.cluster.Bridge;
import org.apache.activemq.artemis.core.server.impl.AckReason;
import org.apache.activemq.artemis.core.transaction.Transaction;
import org.apache.activemq.artemis.spi.core.protocol.RemotingConnection;
import org.apache.activemq.artemis.utils.critical.CriticalComponent;
import org.jboss.logging.BasicLogger;
import org.jboss.logging.Logger;
import org.jboss.logging.annotations.LogMessage;
import org.jboss.logging.annotations.Message;
import org.jboss.logging.annotations.MessageLogger;
/**
* Logger Code 84
*
* each message id must be 6 digits long starting with 84, the 3rd digit donates the level so
*
* INF0 1
* WARN 2
* DEBUG 3
* ERROR 4
* TRACE 5
* FATAL 6
*
* so an INFO message would be 841000 to 841999
*/
@MessageLogger(projectCode = "AMQ")
public interface LoggingActiveMQServerPluginLogger extends BasicLogger {
/**
* The LoggingPlugin logger.
*/
LoggingActiveMQServerPluginLogger LOGGER = Logger.getMessageLogger(LoggingActiveMQServerPluginLogger.class, LoggingActiveMQServerPluginLogger.class.getPackage().getName());
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841000, value = "created connection: {0}", format = Message.Format.MESSAGE_FORMAT)
void afterCreateConnection(RemotingConnection connection);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841001, value = "destroyed connection: {0}", format = Message.Format.MESSAGE_FORMAT)
void afterDestroyConnection(RemotingConnection connection);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841002, value = "created session name: {0}, session connectionID: {1}", format = Message.Format.MESSAGE_FORMAT)
void afterCreateSession(String sessionName, Object sesssionConnectionID);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841003, value = "closed session with session name: {0}, failed: {1}", format = Message.Format.MESSAGE_FORMAT)
void afterCloseSession(String sessionName, boolean sesssionConnectionID);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841004, value = "added session metadata for session name : {0}, key: {1}, data: {2}", format = Message.Format.MESSAGE_FORMAT)
void afterSessionMetadataAdded(String sessionName, String key, String data);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841005, value = "created consumer with ID: {0}, with session name: {1}", format = Message.Format.MESSAGE_FORMAT)
void afterCreateConsumer(String consumerID, String sessionID);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841006, value = "closed consumer ID: {0}, with consumer Session: {1}, failed: {2}", format = Message.Format.MESSAGE_FORMAT)
void afterCloseConsumer(String consumerID, String sessionID, boolean failed);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841007, value = "created queue: {0}", format = Message.Format.MESSAGE_FORMAT)
void afterCreateQueue(Queue queue);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841008, value = "destroyed queue: {0}, with args address: {1}, session: {2}, checkConsumerCount: {3}," +
" removeConsumers: {4}, autoDeleteAddress: {5}", format = Message.Format.MESSAGE_FORMAT)
void afterDestroyQueue(Queue queue,
SimpleString address,
SecurityAuth session,
boolean checkConsumerCount,
boolean removeConsumers,
boolean autoDeleteAddress);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841009, value = "sent message with ID: {0}, session name: {1}, session connectionID: {2}, result: {3}", format = Message.Format.MESSAGE_FORMAT)
void afterSend(String messageID, String sessionName, String sessionConnectionID, RoutingStatus result);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841010, value = "routed message with ID: {0}, result: {1}", format = Message.Format.MESSAGE_FORMAT)
void afterMessageRoute(String messageID, RoutingStatus result);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841011, value = "delivered message with message ID: {0}, consumer info UNAVAILABLE", format = Message.Format.MESSAGE_FORMAT)
void afterDeliverNoConsumer(String messageID);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841012, value = "delivered message with message ID: {0}, to consumer on address: {1}, queue: {2}," +
" consumer sessionID: {3}, consumerID: {4}", format = Message.Format.MESSAGE_FORMAT)
void afterDeliver(String messageID,
SimpleString queueAddress,
SimpleString queueName,
String consumerSessionID,
long consumerID);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841013, value = "expired message: {0}, messageExpiryAddress: {1}", format = Message.Format.MESSAGE_FORMAT)
void messageExpired(MessageReference message, SimpleString messageExpiryAddress);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841014, value = "acknowledged message ID: {0}, with messageRef consumerID: {1}, messageRef QueueName: {2}," +
" with ackReason: {3}", format = Message.Format.MESSAGE_FORMAT)
void messageAcknowledged(String messageID, String consumerID, String queueName, AckReason reason);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841015, value = "deployed bridge: {0}", format = Message.Format.MESSAGE_FORMAT)
void afterDeployBridge(Bridge config);
@LogMessage(level = Logger.Level.INFO)
@Message(id = 841016, value = "criticalFailure called with criticalComponent: {0}", format = Message.Format.MESSAGE_FORMAT)
void criticalFailure(CriticalComponent components);
//DEBUG messages
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843000, value = "beforeCreateSession called with name: {0}, username: {1}, minLargeMessageSize: {2}, connection: {3},"
+ " autoCommitSends: {4}, autoCommitAcks: {5}, preAcknowledge: {6}, xa: {7}, publicAddress: {8}, context: {9}",
format = Message.Format.MESSAGE_FORMAT)
void beforeCreateSession(String name,
String username,
int minLargeMessageSize,
RemotingConnection connection,
boolean autoCommitSends,
boolean autoCommitAcks,
boolean preAcknowledge,
boolean xa,
String publicAddress,
OperationContext context);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843001, value = "beforeCloseSession called with session name : {0}, session: {1}, failed: {2}", format = Message.Format.MESSAGE_FORMAT)
void beforeCloseSession(String sessionName, ServerSession session, boolean failed);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843002, value = "beforeSessionMetadataAdded called with session name: {0} , session: {1}, key: {2}," +
" data: {3}", format = Message.Format.MESSAGE_FORMAT)
void beforeSessionMetadataAdded(String sessionName, ServerSession session, String key, String data);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843003, value = "added session metadata for session name : {0}, session: {1}, key: {2}, data: {3}",
format = Message.Format.MESSAGE_FORMAT)
void afterSessionMetadataAddedDetails(String sessionName, ServerSession session, String key, String data);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843004, value = "beforeCreateConsumer called with ConsumerID: {0}, QueueBinding: {1}, filterString: {2}," +
" browseOnly: {3}, supportLargeMessage: {4}", format = Message.Format.MESSAGE_FORMAT)
void beforeCreateConsumer(String consumerID,
QueueBinding queueBinding,
SimpleString filterString,
boolean browseOnly,
boolean supportLargeMessage);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843005, value = "beforeCloseConsumer called with consumer: {0}, consumer sessionID: {1}, failed: {2}",
format = Message.Format.MESSAGE_FORMAT)
void beforeCloseConsumer(ServerConsumer consumer, String sessionID, boolean failed);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843006, value = "beforeCreateQueue called with queueConfig: {0}", format = Message.Format.MESSAGE_FORMAT)
void beforeCreateQueue(QueueConfig queueConfig);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843007, value = "beforeDestroyQueue called with queueName: {0}, session: {1}, checkConsumerCount: {2}," +
" removeConsumers: {3}, autoDeleteAddress: {4}", format = Message.Format.MESSAGE_FORMAT)
void beforeDestroyQueue(SimpleString queueName,
SecurityAuth session,
boolean checkConsumerCount,
boolean removeConsumers,
boolean autoDeleteAddress);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843008, value = "beforeSend called with message: {0}, tx: {1}, session: {2}, direct: {3}," +
" noAutoCreateQueue: {4}", format = Message.Format.MESSAGE_FORMAT)
void beforeSend(org.apache.activemq.artemis.api.core.Message message,
Transaction tx,
ServerSession session,
boolean direct,
boolean noAutoCreateQueue);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843009, value = "message ID: {0}, message {1}, session name: {2} with tx: {3}, session: {4}, direct: {5}," +
" noAutoCreateQueue: {6}", format = Message.Format.MESSAGE_FORMAT)
void afterSendDetails(String messageID,
org.apache.activemq.artemis.api.core.Message message,
String sessionName,
Transaction tx,
ServerSession session,
boolean direct,
boolean noAutoCreateQueue);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843010, value = "beforeMessageRoute called with message: {0}, context: {1}, direct: {2}, rejectDuplicates: {3}",
format = Message.Format.MESSAGE_FORMAT)
void beforeMessageRoute(org.apache.activemq.artemis.api.core.Message message,
RoutingContext context,
boolean direct,
boolean rejectDuplicates);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843011, value = "afterMessageRoute message: {0}, with context: {1}, direct: {2}, rejectDuplicates: {3}",
format = Message.Format.MESSAGE_FORMAT)
void afterMessageRouteDetails(org.apache.activemq.artemis.api.core.Message message,
RoutingContext context,
boolean direct,
boolean rejectDuplicates);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843012, value = "beforeDeliver called with consumer: {0}, reference: {1}", format = Message.Format.MESSAGE_FORMAT)
void beforeDeliver(ServerConsumer consumer, MessageReference reference);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843013, value = "delivered message with message ID: {0} to consumer on address: {1}, queue: {2}, consumer sessionID: {3}," +
" consumerID: {4}, full message reference: {5}, full consumer: {6}", format = Message.Format.MESSAGE_FORMAT)
void afterDeliverDetails(String messageID,
SimpleString queueAddress,
SimpleString queueName,
String consumerSessionID,
long consumerID,
MessageReference reference,
ServerConsumer consumer);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843014, value = "acknowledged message: {0}, with ackReason: {1}", format = Message.Format.MESSAGE_FORMAT)
void messageAcknowledgedDetails(MessageReference ref, AckReason reason);
@LogMessage(level = Logger.Level.DEBUG)
@Message(id = 843015, value = "beforeDeployBridge called with bridgeConfiguration: {0}", format = Message.Format.MESSAGE_FORMAT)
void beforeDeployBridge(BridgeConfiguration config);
}

View File

@ -126,7 +126,8 @@ and LOG_SENDING_EVENTS will be logged by the broker.
</configuration>
```
The LoggingActiveMQServerPlugin logs information at Log Level `INFO`. By setting the Logger
"org.apache.activemq.artemis.core.server.plugin.impl.LoggingActiveMQServerPlugin" to `DEBUG` more information for each
event will be logged.
Most events in the LoggingActiveMQServerPlugin follow a `beforeX` and `afterX` notification pattern e.g beforeCreateConsumer() and afterCreateConsumer().
At Log Level `INFO`, the LoggingActiveMQServerPlugin logs an entry when an `afterX` notification occurs. By setting the Logger
"org.apache.activemq.artemis.core.server.plugin.impl" to `DEBUG` Level, log entries are generated for both `beforeX` and `afterX` notifications.
Log Level `DEBUG` will also log more information for a notification when available.

View File

@ -50,9 +50,9 @@ public class LoggingActiveMQServerPluginAMQPTest extends LoggingActiveMQServerPl
@Test
@BMRules(rules = {@BMRule(name = "test logAll EVENT",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testLogAll() throws Exception {
//initial plugin
@ -96,9 +96,9 @@ public class LoggingActiveMQServerPluginAMQPTest extends LoggingActiveMQServerPl
@Test
@BMRules(rules = {@BMRule(name = "test LOG_CONSUMER_EVENTS",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testLogConsumerEvents() throws Exception {
ActiveMQServer activeMQServer = createServerWithLoggingPlugin(LoggingActiveMQServerPlugin.LOG_CONSUMER_EVENTS);
@ -142,9 +142,9 @@ public class LoggingActiveMQServerPluginAMQPTest extends LoggingActiveMQServerPl
@Test
@BMRules(rules = {@BMRule(name = "test LOG_SESSION_EVENTS",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testLogSessionEvents() throws Exception {

View File

@ -52,9 +52,9 @@ public class LoggingActiveMQServerPluginOpenWireTest extends LoggingActiveMQServ
@Test
@BMRules(rules = {@BMRule(name = "test queue creation log",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testQueueCreationLog() throws Exception {
//initial plugin
@ -100,9 +100,9 @@ public class LoggingActiveMQServerPluginOpenWireTest extends LoggingActiveMQServ
@Test
@BMRules(rules = {@BMRule(name = "test LOG_SESSION_EVENTS",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testLogSessionEvents() throws Exception {
@ -141,9 +141,9 @@ public class LoggingActiveMQServerPluginOpenWireTest extends LoggingActiveMQServ
@Test
@BMRules(rules = {@BMRule(name = "test logAll EVENT",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testLogAll() throws Exception {
//initial plugin

View File

@ -39,6 +39,7 @@ import org.jboss.byteman.contrib.bmunit.BMRules;
import org.jboss.byteman.contrib.bmunit.BMUnitRunner;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.jboss.logging.Logger.Level;
/**
* NOTE: this test should be run at log level INFO
@ -79,9 +80,9 @@ public class LoggingActiveMQServerPluginTest extends ActiveMQTestBase {
@Test
@BMRules(rules = {@BMRule(name = "test LOG_CONNECTION_EVENTS",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testLogConnectEvents() throws Exception {
ActiveMQServer activeMQServer = createServerWithLoggingPlugin(LoggingActiveMQServerPlugin.LOG_CONNECTION_EVENTS);
@ -130,9 +131,9 @@ public class LoggingActiveMQServerPluginTest extends ActiveMQTestBase {
@Test
@BMRules(rules = {@BMRule(name = "test LOG_SESSION_EVENTS",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testLogSessionEvents() throws Exception {
ActiveMQServer activeMQServer = createServerWithLoggingPlugin(LoggingActiveMQServerPlugin.LOG_SESSION_EVENTS);
@ -166,9 +167,9 @@ public class LoggingActiveMQServerPluginTest extends ActiveMQTestBase {
@Test
@BMRules(rules = {@BMRule(name = "test LOG_CONSUMER_EVENTS",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testLogConsumerEvents() throws Exception {
ActiveMQServer activeMQServer = createServerWithLoggingPlugin(LoggingActiveMQServerPlugin.LOG_CONSUMER_EVENTS);
@ -208,9 +209,9 @@ public class LoggingActiveMQServerPluginTest extends ActiveMQTestBase {
@Test
@BMRules(rules = {@BMRule(name = "test LOG_DELIVERING_EVENTS",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testLogDeliveringEvents() throws Exception {
ActiveMQServer activeMQServer = createServerWithLoggingPlugin(LoggingActiveMQServerPlugin.LOG_DELIVERING_EVENTS);
@ -250,9 +251,9 @@ public class LoggingActiveMQServerPluginTest extends ActiveMQTestBase {
@Test
@BMRules(rules = {@BMRule(name = "test LOG_SENDING_EVENTS",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testLogSendingEvents() throws Exception {
//initial plugin
@ -294,9 +295,9 @@ public class LoggingActiveMQServerPluginTest extends ActiveMQTestBase {
@Test
@BMRules(rules = {@BMRule(name = "test queue creation log",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testQueueCreationLog() throws Exception {
//initial plugin
@ -337,9 +338,9 @@ public class LoggingActiveMQServerPluginTest extends ActiveMQTestBase {
@Test
@BMRules(rules = {@BMRule(name = "test queue creation log",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testExpireMessageLog() throws Exception {
//initial plugin
@ -382,9 +383,9 @@ public class LoggingActiveMQServerPluginTest extends ActiveMQTestBase {
@Test
@BMRules(rules = {@BMRule(name = "test logAll EVENT",
targetClass = "org.jboss.logging.Logger",
targetMethod = "infof",
targetMethod = "logv",
targetLocation = "ENTRY",
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($1, $0)")})
action = "org.apache.activemq.artemis.tests.extras.byteman.LoggingActiveMQServerPluginTest.infoLog($2, $4, $0)")})
public void testLogAll() throws Exception {
//initial plugin
@ -418,42 +419,47 @@ public class LoggingActiveMQServerPluginTest extends ActiveMQTestBase {
}
//collect the log invocation from LoggingActiveMQServerPlugin
public static void infoLog(Object message, org.jboss.logging.Logger logger) {
public static void infoLog(Level level, Object message, org.jboss.logging.Logger logger) {
//not interested
if (!logger.getName().equals(LoggingActiveMQServerPlugin.class.getName())) {
//only interested in log level INFO
if (!level.equals(Level.INFO)) {
return;
}
//only interested in one logger
if (!logger.getName().startsWith(LoggingActiveMQServerPlugin.class.getPackage().getName())) {
return;
}
String stringMessage = (String) message;
if (stringMessage.startsWith("created connection")) {
if (stringMessage.startsWith("AMQ841000")) {
createdConnectionLogs.add(stringMessage);
} else if (stringMessage.startsWith("destroyed connection")) {
} else if (stringMessage.startsWith("AMQ841001")) {
destroyedConnectionLogs.add(stringMessage);
} else if (stringMessage.startsWith("created session")) {
} else if (stringMessage.startsWith("AMQ841002")) {
createdSessionLogs.add(stringMessage);
} else if (stringMessage.startsWith("closed session")) {
} else if (stringMessage.startsWith("AMQ841003")) {
closedSessionLogs.add(stringMessage);
} else if (stringMessage.startsWith("added session metadata")) {
} else if (stringMessage.startsWith("AMQ841004")) {
addedSessionMetaData.add(stringMessage);
} else if (stringMessage.startsWith("created consumer")) {
} else if (stringMessage.startsWith("AMQ841005")) {
createdConsumerLogs.add(stringMessage);
} else if (stringMessage.startsWith("closed consumer")) {
} else if (stringMessage.startsWith("AMQ841006")) {
closedConsumerLogs.add(stringMessage);
} else if (stringMessage.startsWith("delivered message")) {
} else if (stringMessage.startsWith("AMQ841012")) {
deliveredLogs.add(stringMessage);
} else if (stringMessage.startsWith("acknowledged message")) {
} else if (stringMessage.startsWith("AMQ841014")) {
ackedLogs.add(stringMessage);
} else if (stringMessage.startsWith("sent message")) {
} else if (stringMessage.startsWith("AMQ841009")) {
sentLogs.add(stringMessage);
} else if (stringMessage.startsWith("routed message")) {
} else if (stringMessage.startsWith("AMQ841010")) {
routedLogs.add(stringMessage);
} else if (stringMessage.startsWith("created queue")) {
} else if (stringMessage.startsWith("AMQ841007")) {
createdQueueLogs.add(stringMessage);
} else if (stringMessage.startsWith("destroyed queue")) {
} else if (stringMessage.startsWith("AMQ841008")) {
destroyedQueueLogs.add(stringMessage);
} else if (stringMessage.startsWith("expired")) {
} else if (stringMessage.startsWith("AMQ841013")) {
messageExpiredLogs.add(stringMessage);
} else {
unexpectedLogs.add(stringMessage);