You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by ch...@apache.org on 2008/01/17 01:48:04 UTC

svn commit: r612661 - in /activemq/trunk: activemq-console/src/test/resources/ activemq-core/src/main/java/org/apache/activemq/broker/jmx/ activemq-core/src/main/java/org/apache/activemq/transport/ activemq-core/src/main/java/org/apache/activemq/transp...

Author: chirino
Date: Wed Jan 16 16:48:02 2008
New Revision: 612661

URL: http://svn.apache.org/viewvc?rev=612661&view=rev
Log:
Applying https://issues.apache.org/activemq/browse/AMQ-1361

Added:
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/LogWriter.java   (with props)
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControl.java   (with props)
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControlMBean.java   (with props)
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerFactory.java   (with props)
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerView.java   (with props)
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerViewMBean.java   (with props)
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/CustomLogWriter.java   (with props)
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/DefaultLogWriter.java   (with props)
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/util/LogWriterFinder.java   (with props)
    activemq/trunk/activemq-core/src/main/resources/META-INF/services/org/apache/activemq/transport/logwriters/
    activemq/trunk/activemq-core/src/main/resources/META-INF/services/org/apache/activemq/transport/logwriters/custom
    activemq/trunk/activemq-core/src/main/resources/META-INF/services/org/apache/activemq/transport/logwriters/default
Modified:
    activemq/trunk/activemq-console/src/test/resources/activemq.xml
    activemq/trunk/activemq-console/src/test/resources/log4j.properties
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/jmx/BrokerView.java
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/jmx/BrokerViewMBean.java
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLogger.java
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/SslTransportFactory.java
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransport.java
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransportFactory.java
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransportServer.java
    activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/udp/UdpTransportFactory.java
    activemq/trunk/activemq-optional/src/main/java/org/apache/activemq/transport/http/HttpTransportFactory.java

Modified: activemq/trunk/activemq-console/src/test/resources/activemq.xml
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-console/src/test/resources/activemq.xml?rev=612661&r1=612660&r2=612661&view=diff
==============================================================================
--- activemq/trunk/activemq-console/src/test/resources/activemq.xml (original)
+++ activemq/trunk/activemq-console/src/test/resources/activemq.xml Wed Jan 16 16:48:02 2008
@@ -19,10 +19,25 @@
 <beans>
 
   <bean class="org.springframework.beans.factory.config.PropertyPlaceholderConfigurer"/>
-  <broker brokerName="linuxpc3" useJmx="true" xmlns="http://activemq.org/config/1.0" persistent="false" deleteAllMessagesOnStartup="true">
+  
+  <!-- Default configuration -->
+  <broker useJmx="false" xmlns="http://activemq.org/config/1.0">
+	  
+	<transportConnectors>
+      <transportConnector uri="tcp://localhost:61616"/>
+    </transportConnectors>
+	  
+	<persistenceAdapter>
+	  <journaledJDBC journalLogFiles="2" dataDirectory="target/foo"/>
+	</persistenceAdapter>
+	  
+  </broker>
+
+  <!-- Example of broker configuration that uses new logging options and dynamic management of logging
+  <broker useJmx="true" xmlns="http://activemq.org/config/1.0" persistent="false" deleteAllMessagesOnStartup="true">
   
     <transportConnectors>
-      <transportConnector uri="tcp://localhost:61616?trace=true&amp;logWriterName=custom&amp;dynamicManagement=true&amp;startLogging=true&amp;jmxPort=1299"/>
+      <transportConnector uri="tcp://localhost:61616?trace=true&amp;logWriterName=custom&amp;dynamicManagement=true&amp;startLogging=true"/>
     </transportConnectors>
     
     <persistenceAdapter>
@@ -30,6 +45,11 @@
     </persistenceAdapter>
     
   </broker>
+  End of example-->
+  
+<!-- Note: the jmxPort=portnumber option on transportConnectors should only be used on clients.
+On brokers, there is a default port (usually 1099) -->
+  
   
 </beans>
 <!-- END SNIPPET: xbean -->

Modified: activemq/trunk/activemq-console/src/test/resources/log4j.properties
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-console/src/test/resources/log4j.properties?rev=612661&r1=612660&r2=612661&view=diff
==============================================================================
--- activemq/trunk/activemq-console/src/test/resources/log4j.properties (original)
+++ activemq/trunk/activemq-console/src/test/resources/log4j.properties Wed Jan 16 16:48:02 2008
@@ -18,27 +18,37 @@
 #
 # The logging properties used during tests..
 #
-log4j.rootLogger=WARN, B
-
-log4j.additivity.org.apache.activemq.transport.TransportLogger=false
-log4j.logger.org.apache.activemq.transport.TransportLogger=DEBUG, A
+log4j.rootLogger=DEBUG, stdout
+log4j.logger.org.apache.activemq.spring=WARN
 
 # CONSOLE appender, not used by default
 log4j.appender.stdout=org.apache.log4j.ConsoleAppender
 log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
 log4j.appender.stdout.layout.ConversionPattern=%d [%-15.15t] %-5p %-30.30c{1} - %m%n
 
-# File appender for transport level output
-log4j.appender.A=org.apache.log4j.FileAppender
-log4j.appender.A.File=transportlog
-log4j.appender.A.BufferedIO=false
-log4j.appender.A.layout=org.apache.log4j.PatternLayout
-log4j.appender.A.layout.ConversionPattern=%d [%-15.15t] %-5p %-30.30c{1} - %m%n
-
+# Example of properties to make use of new logging options
+#
+#log4j.rootLogger=INFO, A
+#log4j.additivity.org.apache.activemq.transport.TransportLogger=false
+#log4j.logger.org.apache.activemq.transport.TransportLogger=DEBUG, B
+#
+# CONSOLE appender, not used by default
+#log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+#log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+#log4j.appender.stdout.layout.ConversionPattern=%d [%-15.15t] %-5p %-30.30c{1} - %m%n
+#
 # File appender for normal output
-log4j.appender.B=org.apache.log4j.FileAppender
-log4j.appender.B.File=applog
-log4j.appender.B.BufferedIO=false
-log4j.appender.B.layout=org.apache.log4j.PatternLayout
-log4j.appender.B.layout.ConversionPattern=%d [%-15.15t] %-5p %-30.30c{1} - %m%n
-
+#log4j.appender.A=org.apache.log4j.FileAppender
+#log4j.appender.A.File=applog
+#log4j.appender.A.BufferedIO=false
+#log4j.appender.A.layout=org.apache.log4j.PatternLayout
+#log4j.appender.A.layout.ConversionPattern=%d [%-15.15t] %-5p %-30.30c{1} - %m%n
+#
+# File appender for transport level logging output
+#log4j.appender.B=org.apache.log4j.FileAppender
+#log4j.appender.B.File=transportlog
+#log4j.appender.B.BufferedIO=false
+#log4j.appender.B.layout=org.apache.log4j.PatternLayout
+#log4j.appender.B.layout.ConversionPattern=%d [%-15.15t] %-5p %-30.30c{1} - %m%n
+#
+# End of example
\ No newline at end of file

Modified: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/jmx/BrokerView.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/jmx/BrokerView.java?rev=612661&r1=612660&r2=612661&view=diff
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/jmx/BrokerView.java (original)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/jmx/BrokerView.java Wed Jan 16 16:48:02 2008
@@ -16,6 +16,8 @@
  */
 package org.apache.activemq.broker.jmx;
 
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 import java.net.URL;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -32,9 +34,10 @@
 import org.apache.activemq.command.ConsumerInfo;
 import org.apache.activemq.command.RemoveSubscriptionInfo;
 import org.apache.activemq.network.NetworkConnector;
-//import org.apache.log4j.LogManager;
-//import org.apache.log4j.PropertyConfigurator;
 
+/**
+ * @version $Revision$
+ */
 public class BrokerView implements BrokerViewMBean {
 
     final ManagedRegionBroker broker;
@@ -266,16 +269,25 @@
     }
     
     //  doc comment inherited from BrokerViewMBean
-    public void reloadLog4jProperties() throws Exception {
-        /*
-        LogManager.resetConfiguration();
-        ClassLoader cl = this.getClass().getClassLoader();
-        URL log4jprops = cl.getResource("log4j.properties");
-        if (log4jprops != null) {
-            PropertyConfigurator.configure(log4jprops);
+    public void reloadLog4jProperties() throws Throwable {
+
+        // Avoid a direct dependency on log4j.. use reflection.
+        try {
+            ClassLoader cl = getClass().getClassLoader();
+            Class logManagerClass = cl.loadClass("org.apache.log4j.LogManager");
+            
+            Method resetConfiguration = logManagerClass.getMethod("resetConfiguration", new Class[]{});
+            resetConfiguration.invoke(null, new Object[]{});
+            
+            URL log4jprops = cl.getResource("log4j.properties");
+            if (log4jprops != null) {
+                Class propertyConfiguratorClass = cl.loadClass("org.apache.log4j.PropertyConfigurator");
+                Method configure = propertyConfiguratorClass.getMethod("configure", new Class[]{URL.class});
+                configure.invoke(null, new Object[]{log4jprops});
+            }
+        } catch (InvocationTargetException e) {
+            throw e.getTargetException();
         }
-        */
     }
     
-
 }

Modified: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/jmx/BrokerViewMBean.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/jmx/BrokerViewMBean.java?rev=612661&r1=612660&r2=612661&view=diff
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/jmx/BrokerViewMBean.java (original)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/broker/jmx/BrokerViewMBean.java Wed Jan 16 16:48:02 2008
@@ -20,6 +20,10 @@
 
 import org.apache.activemq.Service;
 
+/**
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com (for the reloadLog4jProperties method)
+ * @version $Revision$
+ */
 public interface BrokerViewMBean extends Service {
 
     /**
@@ -159,8 +163,8 @@
     /**
      * Reloads log4j.properties from the classpath.
      * This methods calls org.apache.activemq.transport.TransportLoggerControl.reloadLog4jProperties
-     * @throws Exception
+     * @throws Throwable 
      */
-    public void reloadLog4jProperties() throws Exception;
+    public void reloadLog4jProperties() throws Throwable;
     
 }

Added: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/LogWriter.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/LogWriter.java?rev=612661&view=auto
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/LogWriter.java (added)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/LogWriter.java Wed Jan 16 16:48:02 2008
@@ -0,0 +1,85 @@
+/**
+ * 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.transport;
+
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+
+/**
+ * Interface for classes that will be called by the TransportLogger
+ * class to actually write to a log file.
+ * Every class that implements this interface has do be declared in
+ * the resources/META-INF/services/org/apache/activemq/transport/logwriters
+ * directory, by creating a file with the name of the writer (for example
+ * "default") and including the line
+ * class=org.apache.activemq.transport.logwriters.(Name of the LogWriter class)
+ * 
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com
+ * @version $Revision$
+ */
+public interface LogWriter {
+
+    /**
+     * Writes a header message to the log.
+     * @param log The log to be written to.
+     */
+    public void initialMessage(Log log);
+    
+    /**
+     * Writes a message to a log when a request command is sent.
+     * @param log The log to be written to.
+     * @param command The command to be logged.
+     */
+    public void logRequest (Log log, Object command);
+    
+    /**
+     * Writes a message to a log when a response command is received.
+     * @param log The log to be written to.
+     * @param command The command to be logged.
+     */
+    public void logResponse (Log log, Object response);
+
+    /**
+     * Writes a message to a log when an asynchronous equest command is sent.
+     * @param log The log to be written to.
+     * @param command The command to be logged.
+     */
+    public void logAsyncRequest (Log log, Object command);
+    
+    /**
+     * Writes a message to a log when message is sent.
+     * @param log The log to be written to.
+     * @param command The command to be logged.
+     */
+    public void logOneWay (Log log, Object command);
+    
+    /**
+     * Writes a message to a log when message is received.
+     * @param log The log to be written to.
+     * @param command The command to be logged.
+     */
+    public void logReceivedCommand (Log log, Object command);
+    
+    /**
+     * Writes a message to a log when an exception is received.
+     * @param log The log to be written to.
+     * @param command The command to be logged.
+     */
+    public void logReceivedException (Log log, IOException error);
+    
+}

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/LogWriter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/LogWriter.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLogger.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLogger.java?rev=612661&r1=612660&r2=612661&view=diff
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLogger.java (original)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLogger.java Wed Jan 16 16:48:02 2008
@@ -19,71 +19,142 @@
 import java.io.IOException;
 
 import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 
 /**
+ * This TransportFilter implementation writes output to a log
+ * as it intercepts commands / events before sending them to the
+ * following layer in the Transport stack.
+ * 
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com
  * @version $Revision$
  */
 public class TransportLogger extends TransportFilter {
 
-    private static int lastId;
     private final Log log;
-
-    public TransportLogger(Transport next) {
-        this(next, LogFactory.getLog(TransportLogger.class.getName() + ".Connection:" + getNextId()));
-    }
-
-    public TransportLogger(Transport next, Log log) {
+    private boolean logging;
+    private final LogWriter logWriter;
+    private TransportLoggerView view;
+
+    public TransportLogger(Transport next, Log log, boolean startLogging, LogWriter logWriter) {
+        // Changed constructor to pass the implementation of the LogWriter interface
+        // that will be used to write the messages.
         super(next);
         this.log = log;
+        this.logging = startLogging;
+        this.logWriter = logWriter;
     }
 
-    private static synchronized int getNextId() {
-        return ++lastId;
-    }
+    /**
+     * Returns true if logging is activated for this TransportLogger, false otherwise.
+     * @return true if logging is activated for this TransportLogger, false otherwise.
+     */
+    public boolean isLogging() {
+        return logging;
+    }
+
+    /**
+     * Sets if logging should be activated for this TransportLogger.
+     * @param logging true to activate logging, false to deactivate.
+     */
+    public void setLogging(boolean logging) {
+        this.logging = logging;
+    } 
 
     public Object request(Object command) throws IOException {
-        log.debug("SENDING REQUEST: " + command);
+        // Changed this method to use a LogWriter object to actually 
+        // print the messages to the log, and only in case of logging 
+        // being active, instead of logging the message directly.
+        if (logging)
+            logWriter.logRequest(log, command);
         Object rc = super.request(command);
-        log.debug("GOT RESPONSE: " + rc);
+        if (logging)
+            logWriter.logResponse(log, command);
         return rc;
     }
 
     public Object request(Object command, int timeout) throws IOException {
-        log.debug("SENDING REQUEST: " + command);
+        // Changed this method to use a LogWriter object to actually 
+        // print the messages to the log, and only in case of logging 
+        // being active, instead of logging the message directly.
+        if (logging)
+            logWriter.logRequest(log, command);
         Object rc = super.request(command, timeout);
-        log.debug("GOT RESPONSE: " + rc);
+        if (logging)
+            logWriter.logResponse(log, command);
         return rc;
     }
 
     public FutureResponse asyncRequest(Object command, ResponseCallback responseCallback) throws IOException {
-        log.debug("SENDING ASNYC REQUEST: " + command);
+        // Changed this method to use a LogWriter object to actually 
+        // print the messages to the log, and only in case of logging 
+        // being active, instead of logging the message directly.
+        if (logging)
+            logWriter.logAsyncRequest(log, command);
         FutureResponse rc = next.asyncRequest(command, responseCallback);
         return rc;
     }
 
     public void oneway(Object command) throws IOException {
-        if (log.isDebugEnabled()) {
-            log.debug("SENDING: " + command);
+        // Changed this method to use a LogWriter object to actually 
+        // print the messages to the log, and only in case of logging 
+        // being active, instead of logging the message directly.
+        if( logging && log.isDebugEnabled() ) {
+            logWriter.logOneWay(log, command);
         }
         next.oneway(command);
     }
 
     public void onCommand(Object command) {
-        if (log.isDebugEnabled()) {
-            log.debug("RECEIVED: " + command);
+        // Changed this method to use a LogWriter object to actually 
+        // print the messages to the log, and only in case of logging 
+        // being active, instead of logging the message directly.
+        if( logging && log.isDebugEnabled() ) {
+            logWriter.logReceivedCommand(log, command);
         }
         getTransportListener().onCommand(command);
     }
 
     public void onException(IOException error) {
-        if (log.isDebugEnabled()) {
-            log.debug("RECEIVED Exception: " + error, error);
+        // Changed this method to use a LogWriter object to actually 
+        // print the messages to the log, and only in case of logging 
+        // being active, instead of logging the message directly.
+        if( logging && log.isDebugEnabled() ) {
+            logWriter.logReceivedException(log, error);
         }
         getTransportListener().onException(error);
     }
 
+    /**
+     * Gets the associated MBean for this TransportLogger.
+     * @return the associated MBean for this TransportLogger.
+     */
+    public TransportLoggerView getView() {
+        return view;
+    }
+
+    /**
+     * Sets the associated MBean for this TransportLogger.
+     * @param view the associated MBean for this TransportLogger.
+     */
+    public void setView(TransportLoggerView view) {
+        this.view = view;
+    }
+
+
     public String toString() {
         return next.toString();
     }
+
+
+    /**
+     * We need to override this method
+     * so that we can unregister the associated
+     * MBean to avoid a memory leak.
+     */
+    public void finalize() throws Throwable {
+        if (view != null) {
+            view.unregister();    
+        }
+    }
+
 }

Added: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControl.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControl.java?rev=612661&view=auto
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControl.java (added)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControl.java Wed Jan 16 16:48:02 2008
@@ -0,0 +1,52 @@
+/**
+ * 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.transport;
+
+import org.apache.activemq.broker.jmx.BrokerView;
+import org.apache.activemq.broker.jmx.ManagementContext;
+
+/**
+ * Implementation of the TransportLoggerControlMBean interface,
+ * which is an MBean used to control all TransportLoggers at once.
+ * 
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com
+ * @version $Revision$
+ */
+public class TransportLoggerControl implements TransportLoggerControlMBean {
+
+    /**
+     * Constructor
+     */
+    public TransportLoggerControl(ManagementContext managementContext) {
+    }
+
+    // doc comment inherited from TransportLoggerControlMBean
+    public void disableAllTransportLoggers() {
+        TransportLoggerView.disableAllTransportLoggers();
+    }
+
+    // doc comment inherited from TransportLoggerControlMBean
+    public void enableAllTransportLoggers() {
+        TransportLoggerView.enableAllTransportLoggers();
+    }
+
+    //  doc comment inherited from TransportLoggerControlMBean
+    public void reloadLog4jProperties() throws Throwable {
+        new BrokerView(null, null).reloadLog4jProperties();
+    }
+
+}

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControl.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControl.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControlMBean.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControlMBean.java?rev=612661&view=auto
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControlMBean.java (added)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControlMBean.java Wed Jan 16 16:48:02 2008
@@ -0,0 +1,46 @@
+/**
+ * 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.transport;
+
+/**
+ * MBean used to manage all of the TransportLoggers at once.
+ * Avalaible operations:
+ *  -Enable logging for all TransportLoggers at once.
+ *  -Disable logging for all TransportLoggers at once.
+ *  
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com
+ * @version $Revision$
+ */
+public interface TransportLoggerControlMBean {
+
+    /**
+     * Enable logging for all Transport Loggers at once.
+     */
+    public void enableAllTransportLoggers();
+
+    /**
+     * Disable logging for all Transport Loggers at once.
+     */
+    public void disableAllTransportLoggers();
+
+    /**
+     * Reloads log4j.properties from the classpath
+     * @throws Throwable 
+     */
+    public void reloadLog4jProperties() throws Throwable;
+
+}

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControlMBean.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerControlMBean.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerFactory.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerFactory.java?rev=612661&view=auto
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerFactory.java (added)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerFactory.java Wed Jan 16 16:48:02 2008
@@ -0,0 +1,213 @@
+/**
+ * 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.transport;
+
+import java.io.IOException;
+
+import javax.management.ObjectName;
+
+import org.apache.activemq.broker.jmx.ManagementContext;
+import org.apache.activemq.util.IOExceptionSupport;
+import org.apache.activemq.util.LogWriterFinder;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Singleton class to create TransportLogger objects.
+ * When the method getInstance() is called for the first time,
+ * a TransportLoggerControlMBean is created and registered.
+ * This MBean permits enabling and disabling the logging for
+ * all TransportLogger objects at once.
+ * 
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com
+ * @version $Revision$
+ * @see TransportLoggerControlMBean
+ */
+public class TransportLoggerFactory {
+
+    private static final Log LOG = LogFactory.getLog(TransportLoggerFactory.class);
+
+    private static TransportLoggerFactory instance;
+    private static int lastId=0;
+    private static final LogWriterFinder logWriterFinder = new LogWriterFinder("META-INF/services/org/apache/activemq/transport/logwriters/");
+
+    /**
+     * LogWriter that will be used if none is specified.
+     */
+    public static String defaultLogWriterName = "default";
+    /**
+     * If transport logging is enabled, it will be possible to control
+     * the transport loggers or not based on this value 
+     */
+    private static boolean defaultDynamicManagement = false;
+    /**
+     * If transport logging is enabled, the transport loggers will initially
+     * output or not depending on this value.
+     * This setting only has a meaning if 
+     */
+    private static boolean defaultInitialBehavior = true;
+    /**
+     * Default port to control the transport loggers through JMX
+     */
+    private static int defaultJmxPort = 1099;
+
+    private boolean transportLoggerControlCreated = false;
+    private ManagementContext managementContext;
+    private ObjectName objectName;
+
+    /**
+     * Private constructor.
+     */
+    private TransportLoggerFactory() {
+    }
+
+    /**
+     * Returns a TransportLoggerFactory object which can be used to create TransportLogger objects.
+     * @return a TransportLoggerFactory object
+     */
+    public static synchronized TransportLoggerFactory getInstance() {
+        if (instance == null) {
+            instance = new TransportLoggerFactory();
+        }
+        return instance;
+    }
+
+    public void stop() {
+        try {
+            if (this.transportLoggerControlCreated) {
+                this.managementContext.unregisterMBean(this.objectName);
+                this.managementContext.stop();
+                this.managementContext = null;
+            }
+        } catch (Exception e) {
+            LOG.error("TransportLoggerFactory could not be stopped, reason: " + e, e);
+        }
+
+    }
+
+    /**
+     * Creates a TransportLogger object, that will be inserted in the Transport Stack.
+     * Uses the default initial behavior, the default log writer, and creates a new
+     * log4j object to be used by the TransportLogger.
+     * @param next The next Transport layer in the Transport stack.
+     * @return A TransportLogger object.
+     * @throws IOException
+     */
+    public TransportLogger createTransportLogger(Transport next) throws IOException {
+        int id = getNextId();
+        return createTransportLogger(next, id, createLog(id), defaultLogWriterName, defaultDynamicManagement, defaultInitialBehavior, defaultJmxPort);
+    }
+    
+    /**
+     * Creates a TransportLogger object, that will be inserted in the Transport Stack.
+     * Uses the default initial behavior and the default log writer.
+     * @param next The next Transport layer in the Transport stack.
+     * @param log The log4j log that will be used by the TransportLogger.
+     * @return A TransportLogger object.
+     * @throws IOException
+     */
+    public TransportLogger createTransportLogger(Transport next, Log log) throws IOException {
+        return createTransportLogger(next, getNextId(), log, defaultLogWriterName, defaultDynamicManagement, defaultInitialBehavior, defaultJmxPort);
+    }
+
+    /**
+     * Creates a TransportLogger object, that will be inserted in the Transport Stack.
+     * Creates a new log4j object to be used by the TransportLogger.
+     * @param next The next Transport layer in the Transport stack.
+     * @param startLogging Specifies if this TransportLogger should be initially active or not.
+     * @param logWriterName The name or the LogWriter to be used. Different log writers can output
+     * logs with a different format.
+     * @return A TransportLogger object.
+     * @throws IOException
+     */
+    public TransportLogger createTransportLogger(Transport next, String logWriterName,
+            boolean useJmx, boolean startLogging, int jmxport) throws IOException {
+        int id = getNextId();
+        return createTransportLogger(next, id, createLog(id), logWriterName, useJmx, startLogging, jmxport);
+    }
+
+
+
+    /**
+     * Creates a TransportLogger object, that will be inserted in the Transport Stack.
+     * @param next The next Transport layer in the Transport stack.
+     * @param id The id of the transport logger.
+     * @param log The log4j log that will be used by the TransportLogger.
+     * @param logWriterName The name or the LogWriter to be used. Different log writers can output
+     * @param dynamicManagement Specifies if JMX will be used to switch on/off the TransportLogger to be created.
+     * @param startLogging Specifies if this TransportLogger should be initially active or not. Only has a meaning if
+     * dynamicManagement = true.
+     * @param jmxPort the port to be used by the JMX server. It should only be different from 1099 (broker's default JMX port)
+     * when it's a client that is using Transport Logging. In a broker, if the port is different from 1099, 2 JMX servers will
+     * be created, both identical, with all the MBeans.
+     * @return A TransportLogger object.
+     * @throws IOException
+     */
+    public TransportLogger createTransportLogger(Transport next, int id, Log log,
+            String logWriterName, boolean dynamicManagement, boolean startLogging, int jmxport) throws IOException {
+        try {
+            LogWriter logWriter = logWriterFinder.newInstance(logWriterName);
+            TransportLogger tl =  new TransportLogger (next, log, startLogging, logWriter);
+            if (dynamicManagement) {
+                synchronized (this) {
+                    if (!this.transportLoggerControlCreated) {
+                        this.createTransportLoggerControl(jmxport);
+                    }
+                }
+                TransportLoggerView tlv = new TransportLoggerView(tl, next.toString(), id, this.managementContext);
+                tl.setView(tlv);
+            }
+            return tl;
+        } catch (Throwable e) {
+            throw IOExceptionSupport.create("Could not create log writer object for: " + logWriterName + ", reason: " + e, e);
+        }
+    }
+
+    synchronized private static int getNextId() {
+        return ++lastId;
+    }
+
+    private static Log createLog(int id) {
+        return LogFactory.getLog(TransportLogger.class.getName()+".Connection:" + id);
+    }
+    
+    /**
+     * Starts the management context.
+     * Creates and registers a TransportLoggerControl MBean which enables the user
+     * to enable/disable logging for all transport loggers at once.
+     */
+     private void createTransportLoggerControl(int port) {
+         try {
+             this.managementContext = new ManagementContext();
+             this.managementContext.setConnectorPort(port);
+             this.managementContext.start();
+         } catch (Exception e) {
+             LOG.error("Management context could not be started, reason: " + e, e);
+         }
+
+         try {
+             this.objectName = new ObjectName(this.managementContext.getJmxDomainName()+":"+ "Type=TransportLoggerControl");
+             this.managementContext.getMBeanServer().registerMBean(new TransportLoggerControl(this.managementContext),this.objectName);
+             
+             this.transportLoggerControlCreated = true;
+
+         } catch (Exception e) {
+             LOG.error("TransportLoggerControlMBean could not be registered, reason: " + e, e);
+         }
+     }
+
+}

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerFactory.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerFactory.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerView.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerView.java?rev=612661&view=auto
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerView.java (added)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerView.java Wed Jan 16 16:48:02 2008
@@ -0,0 +1,173 @@
+/**
+ * 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.transport;
+
+import java.lang.ref.WeakReference;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import javax.management.ObjectName;
+
+import org.apache.activemq.broker.jmx.ManagementContext;
+import org.apache.activemq.util.JMXSupport;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Class implementing the TransportLoggerViewMBean interface.
+ * When an object of this class is created, it registers itself in
+ * the MBeanServer of the management context provided.
+ * When a TransportLogger object is finalized because the Transport Stack
+ * where it resides is no longer in use, the method unregister() will be called.
+ * 
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com
+ * @version $Revision$ 
+ * @see TransportLoggerViewMBean.
+ */
+public class TransportLoggerView implements TransportLoggerViewMBean {
+
+    private static final Log log = LogFactory.getLog(TransportLoggerView.class);
+    
+    /**
+     * Set with the TransportLoggerViews objects created.
+     * Used by the methods enableAllTransportLoggers and diablellTransportLoggers.
+     * The method unregister() removes objects from this set.
+     */
+    private static Set<TransportLoggerView> transportLoggerViews = Collections.synchronizedSet(new HashSet<TransportLoggerView>());
+
+    private final WeakReference<TransportLogger> transportLogger;
+    private final String nextTransportName;
+    private final int id;
+    private final ManagementContext managementContext;
+    private final ObjectName name;
+
+    /**
+     * Constructor.
+     * @param transportLogger The TransportLogger object which is to be managed by this MBean.
+     * @param nextTransportName The name of the next TransportLayer. This is used to give a unique
+     * name for each MBean of the TransportLoggerView class.
+     * @param id The id of the TransportLogger to be watched.
+     * @param managementContext The management context who has the MBeanServer where this MBean will be registered.
+     */
+    public TransportLoggerView (TransportLogger transportLogger, String nextTransportName, int id, ManagementContext managementContext) {
+        this.transportLogger = new WeakReference<TransportLogger>(transportLogger);
+        this.nextTransportName = nextTransportName;
+        this.id = id;
+        this.managementContext = managementContext;
+        this.name = this.createTransportLoggerObjectName();
+        
+        TransportLoggerView.transportLoggerViews.add(this);
+        this.register();
+    }
+    
+    /**
+     * Enable logging for all Transport Loggers at once.
+     */
+    public static void enableAllTransportLoggers() {
+        for (TransportLoggerView view : transportLoggerViews) {
+            view.enableLogging();
+        }
+    }
+    
+    /**
+     * Disable logging for all Transport Loggers at once.
+     */
+    public static void disableAllTransportLoggers() {
+        for (TransportLoggerView view : transportLoggerViews) {
+            view.disableLogging();
+        }
+    }
+
+    // doc comment inherited from TransportLoggerViewMBean
+    public void enableLogging() {
+        this.setLogging(true);
+    }
+
+    // doc comment inherited from TransportLoggerViewMBean
+    public void disableLogging() {
+        this.setLogging(false);
+    }   
+
+    // doc comment inherited from TransportLoggerViewMBean
+    public boolean isLogging() {
+        return transportLogger.get().isLogging();
+    }
+
+    // doc comment inherited from TransportLoggerViewMBean
+    public void setLogging(boolean logging) {
+        transportLogger.get().setLogging(logging);
+    }
+
+    /**
+     * Registers this MBean in the MBeanServer of the management context
+     * provided at creation time. This method is only called by the constructor.
+     */
+    private void register() {
+        try {
+            this.managementContext.getMBeanServer().registerMBean(this, this.name);
+        } catch (Exception e) {
+            log.error("Could not register MBean for TransportLoggerView " + id + "with name " + this.name.toString() + ", reason: " + e, e);
+        }
+
+    }
+
+    /**
+     * Unregisters the MBean from the MBeanServer of the management context
+     * provided at creation time.
+     * This method is called by the TransportLogger object being managed when
+     * the TransportLogger object is finalized, to avoid the memory leak that
+     * would be caused if MBeans were not unregistered. 
+     */
+    public void unregister() {
+        
+        TransportLoggerView.transportLoggerViews.remove(this);
+        
+        try {
+            this.managementContext.getMBeanServer().unregisterMBean(this.name);
+        } catch (Exception e) {
+            log.error("Could not unregister MBean for TransportLoggerView " + id + "with name " + this.name.toString() + ", reason: " + e, e);
+        }
+    }
+
+    /**
+     * Creates the ObjectName to be used when registering the MBean.
+     * @return the ObjectName to be used when registering the MBean.
+     */
+    private ObjectName createTransportLoggerObjectName()  {
+        try {
+            return new ObjectName(
+                    createTransportLoggerObjectNameRoot(this.managementContext)
+                    + JMXSupport.encodeObjectNamePart(TransportLogger.class.getSimpleName()
+                            + " " + this.id + ";" + this.nextTransportName));
+        } catch (Exception e) {
+            log.error("Could not create ObjectName for TransportLoggerView " + id + ", reason: " + e, e);
+            return null;
+        }
+    }
+
+    /**
+     * Creates the part of the ObjectName that will be used by all MBeans.
+     * This method is public so it can be used by the TransportLoggerControl class.
+     * @param managementContext
+     * @return A String with the part of the ObjectName common to all the TransportLoggerView MBeans.
+     */
+    public static String createTransportLoggerObjectNameRoot(ManagementContext managementContext) {
+        return managementContext.getJmxDomainName()+":"+"Type=TransportLogger,"+"TransportLoggerName=";
+    }
+
+}

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerView.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerView.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerViewMBean.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerViewMBean.java?rev=612661&view=auto
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerViewMBean.java (added)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerViewMBean.java Wed Jan 16 16:48:02 2008
@@ -0,0 +1,56 @@
+/**
+ * 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.transport;
+
+/**
+ * MBean to manage a single Transport Logger.
+ * It can inform if the logger is currently writing to a log file or not,
+ * by setting the logging property or by using the operations
+ * enableLogging() and disableLogging().
+ * 
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com
+ * @version $Revision$
+ */
+public interface TransportLoggerViewMBean {
+
+    /**
+     * Returns if the managed TransportLogger is currently active
+     * (writing to a log) or not.
+     * @return if the managed TransportLogger is currently active
+     * (writing to a log) or not.
+     */
+    public boolean isLogging();
+    
+    /**
+     * Enables or disables logging for the managed TransportLogger.
+     * @param logging Boolean value to enable or disable logging for
+     * the managed TransportLogger.
+     * true to enable logging, false to disable logging.
+     */
+    public void setLogging(boolean logging);
+    
+    /**
+     * Enables logging for the managed TransportLogger.
+     */
+    public void enableLogging();
+    
+    /**
+     * Disables logging for the managed TransportLogger.
+     */
+    public void disableLogging();
+    
+}

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerViewMBean.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/TransportLoggerViewMBean.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/CustomLogWriter.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/CustomLogWriter.java?rev=612661&view=auto
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/CustomLogWriter.java (added)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/CustomLogWriter.java Wed Jan 16 16:48:02 2008
@@ -0,0 +1,161 @@
+/**
+ * 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.transport.logwriters;
+
+import java.io.IOException;
+
+import org.apache.activemq.command.BaseCommand;
+import org.apache.activemq.command.ConnectionInfo;
+import org.apache.activemq.command.Message;
+import org.apache.activemq.command.MessageAck;
+import org.apache.activemq.command.MessageDispatch;
+import org.apache.activemq.command.ProducerAck;
+import org.apache.activemq.command.ProducerId;
+import org.apache.activemq.command.WireFormatInfo;
+import org.apache.activemq.transport.LogWriter;
+import org.apache.commons.logging.Log;
+
+/**
+ * Custom implementation of LogWriter interface.
+ * 
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com
+ * @version $Revision$
+ */
+public class CustomLogWriter implements LogWriter {
+    
+    // doc comment inherited from LogWriter
+    public void initialMessage(Log log) {
+        
+    }
+
+    // doc comment inherited from LogWriter
+    public void logRequest (Log log, Object command) {
+        log.debug("$$ SENDREQ: " + CustomLogWriter.commandToString(command));
+    }
+    
+    // doc comment inherited from LogWriter
+    public void logResponse (Log log, Object response) {
+        log.debug("$$ GOT_RESPONSE: "+response);
+    }
+    
+    // doc comment inherited from LogWriter
+    public void logAsyncRequest (Log log, Object command) {
+        log.debug("$$ SENDING_ASNYC_REQUEST: "+command);
+    }
+    
+    // doc comment inherited from LogWriter
+    public void logOneWay (Log log, Object command) {
+        log.debug("$$ SENDING: " + CustomLogWriter.commandToString(command));
+    }
+    
+    // doc comment inherited from LogWriter
+    public void logReceivedCommand (Log log, Object command) {
+        log.debug("$$ RECEIVED: " + CustomLogWriter.commandToString(command));
+    }
+    
+    // doc comment inherited from LogWriter
+    public void logReceivedException (Log log, IOException error) {
+        log.debug("$$ RECEIVED_EXCEPTION: "+error, error);
+    }
+    
+    /**
+     * Transforms a command into a String
+     * @param command An object (hopefully of the BaseCommand class or subclass)
+     * to be transformed into String.
+     * @return A String which will be written by the CustomLogWriter.
+     * If the object is not a BaseCommand, the String 
+     * "Unrecognized_object " + command.toString()
+     * will be returned.
+     */
+    private static String commandToString(Object command) {
+        StringBuilder sb = new StringBuilder();
+        
+        if (command instanceof BaseCommand) {
+
+            BaseCommand bc = (BaseCommand)command;
+            sb.append(command.getClass().getSimpleName());
+            sb.append(' ');
+            sb.append(bc.isResponseRequired() ? 'T' : 'F');
+            
+            
+            Message m = null;
+            
+            if (bc instanceof Message) {
+                m = (Message)bc;
+            }
+            if (bc instanceof MessageDispatch){
+                m = ((MessageDispatch)bc).getMessage();   
+            }
+                
+            if (m != null) {
+                sb.append(' ');
+                sb.append(m.getMessageId());
+                sb.append(',');
+                sb.append(m.getCommandId());
+                ProducerId pid = m.getProducerId();
+                long sid = pid.getSessionId();
+                sb.append(',');
+                sb.append(pid.getConnectionId());
+                sb.append(',');
+                sb.append(sid);
+                sb.append(',');
+                sb.append(pid.getValue());
+                sb.append(',');
+                sb.append(m.getCorrelationId());
+                sb.append(',');
+                sb.append(m.getType());
+            }
+            
+            if (bc instanceof MessageDispatch){
+                sb.append(" toConsumer:");
+                sb.append(((MessageDispatch)bc).getConsumerId());
+            }
+            
+            if (bc instanceof ProducerAck) {
+                sb.append(" ProducerId:");
+                sb.append(((ProducerAck)bc).getProducerId());
+            }
+            
+            if (bc instanceof MessageAck) {
+                MessageAck ma = (MessageAck)bc;
+                sb.append(" ConsumerID:");
+                sb.append(ma.getConsumerId());
+                sb.append(" ack:");
+                sb.append(ma.getFirstMessageId());
+                sb.append('-');
+                sb.append(ma.getLastMessageId());
+            }
+            
+            if (bc instanceof ConnectionInfo) {
+                ConnectionInfo ci = (ConnectionInfo)bc;
+                
+                sb.append(' ');
+                sb.append(ci.getConnectionId());
+            }
+            
+        } else if (command instanceof WireFormatInfo){
+            sb.append("WireFormatInfo");
+            
+        } else {
+            sb.append("Unrecognized_object ");
+            sb.append(command.toString());
+        }
+        
+        return sb.toString();
+    }
+
+}

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/CustomLogWriter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/CustomLogWriter.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/DefaultLogWriter.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/DefaultLogWriter.java?rev=612661&view=auto
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/DefaultLogWriter.java (added)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/DefaultLogWriter.java Wed Jan 16 16:48:02 2008
@@ -0,0 +1,69 @@
+/**
+ * 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.transport.logwriters;
+
+import java.io.IOException;
+
+import org.apache.activemq.transport.LogWriter;
+import org.apache.commons.logging.Log;
+
+/**
+ * Implementation of LogWriter interface to keep ActiveMQ's
+ * old logging format.
+ * 
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com
+ * @version $Revision$
+ */
+public class DefaultLogWriter implements LogWriter {
+
+    // doc comment inherited from LogWriter
+    public void initialMessage(Log log) {
+        // Default log writer does nothing here
+    }
+
+    // doc comment inherited from LogWriter
+    public void logRequest (Log log, Object command) {
+        log.debug("SENDING REQUEST: "+command);
+    }
+
+    // doc comment inherited from LogWriter
+    public void logResponse (Log log, Object response) {
+        log.debug("GOT RESPONSE: "+response);
+    }
+
+    // doc comment inherited from LogWriter
+    public void logAsyncRequest (Log log, Object command) {
+        log.debug("SENDING ASNYC REQUEST: "+command);
+    }
+
+    // doc comment inherited from LogWriter
+    public void logOneWay (Log log, Object command) {
+        log.debug("SENDING: "+command);
+    }
+
+    // doc comment inherited from LogWriter
+    public void logReceivedCommand (Log log, Object command) {
+        log.debug("RECEIVED: " + command);
+    }
+
+    // doc comment inherited from LogWriter
+    public void logReceivedException (Log log, IOException error) {
+        log.debug("RECEIVED Exception: "+error, error);
+    }
+
+
+}

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/DefaultLogWriter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/logwriters/DefaultLogWriter.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Modified: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/SslTransportFactory.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/SslTransportFactory.java?rev=612661&r1=612660&r2=612661&view=diff
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/SslTransportFactory.java (original)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/SslTransportFactory.java Wed Jan 16 16:48:02 2008
@@ -37,7 +37,7 @@
 import org.apache.activemq.openwire.OpenWireFormat;
 import org.apache.activemq.transport.InactivityMonitor;
 import org.apache.activemq.transport.Transport;
-import org.apache.activemq.transport.TransportLogger;
+import org.apache.activemq.transport.TransportLoggerFactory;
 import org.apache.activemq.transport.TransportServer;
 import org.apache.activemq.transport.WireFormatNegotiator;
 import org.apache.activemq.util.IOExceptionSupport;
@@ -54,7 +54,8 @@
  * factory will have their needClientAuth option set to false.
  * 
  * @author sepandm@gmail.com (Sepand)
- * @version $Revision: $
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com (logging improvement modifications)
+ * @version $Revision$
  */
 public class SslTransportFactory extends TcpTransportFactory {
     // The log this uses.,
@@ -104,7 +105,12 @@
         sslTransport.setSocketOptions(socketOptions);
 
         if (sslTransport.isTrace()) {
-            transport = new TransportLogger(transport);
+            try {
+                transport = TransportLoggerFactory.getInstance().createTransportLogger(transport,
+                        sslTransport.getLogWriterName(), sslTransport.isDynamicManagement(), sslTransport.isStartLogging(), sslTransport.getJmxPort());
+            } catch (Throwable e) {
+                LOG.error("Could not create TransportLogger object for: " + sslTransport.getLogWriterName() + ", reason: " + e, e);
+            }
         }
 
         transport = new InactivityMonitor(transport);

Modified: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransport.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransport.java?rev=612661&r1=612660&r2=612661&view=diff
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransport.java (original)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransport.java Wed Jan 16 16:48:02 2008
@@ -40,6 +40,7 @@
 
 import org.apache.activemq.Service;
 import org.apache.activemq.transport.Transport;
+import org.apache.activemq.transport.TransportLoggerFactory;
 import org.apache.activemq.transport.TransportThreadSupport;
 import org.apache.activemq.util.IntrospectionSupport;
 import org.apache.activemq.util.ServiceStopper;
@@ -50,6 +51,7 @@
 /**
  * An implementation of the {@link Transport} interface using raw tcp/ip
  * 
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com (logging improvement modifications)
  * @version $Revision$
  */
 public class TcpTransport extends TransportThreadSupport implements Transport, Service, Runnable {
@@ -75,7 +77,12 @@
      * This parameter is most probably set in Connection or TransportConnector URIs.
      */
     protected boolean trace = false;
-    
+    /**
+     * Name of the LogWriter implementation to use.
+     * Names are mapped to classes in the resources/META-INF/services/org/apache/activemq/transport/logwriters directory.
+     * This parameter is most probably set in Connection or TransportConnector URIs.
+     */
+    protected String logWriterName = TransportLoggerFactory.defaultLogWriterName;
     /**
      * Specifies if the TransportLogger will be manageable by JMX or not.
      * Also, as long as there is at least 1 TransportLogger which is manageable,
@@ -206,12 +213,19 @@
     public void setTrace(boolean trace) {
         this.trace = trace;
     }
+    
+    public String getLogWriterName() {
+        return logWriterName;
+    }
+
+    public void setLogWriterName(String logFormat) {
+        this.logWriterName = logFormat;
+    }
 
     public boolean isDynamicManagement() {
         return dynamicManagement;
     }
 
-
     public void setDynamicManagement(boolean useJmx) {
         this.dynamicManagement = useJmx;
     }
@@ -220,22 +234,18 @@
         return startLogging;
     }
 
-
     public void setStartLogging(boolean startLogging) {
         this.startLogging = startLogging;
     }
 
-
     public int getJmxPort() {
         return jmxPort;
     }
 
-
     public void setJmxPort(int jmxPort) {
         this.jmxPort = jmxPort;
     }
-
-
+    
     public int getMinmumWireFormatVersion() {
         return minmumWireFormatVersion;
     }

Modified: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransportFactory.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransportFactory.java?rev=612661&r1=612660&r2=612661&view=diff
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransportFactory.java (original)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransportFactory.java Wed Jan 16 16:48:02 2008
@@ -30,7 +30,7 @@
 import org.apache.activemq.transport.InactivityMonitor;
 import org.apache.activemq.transport.Transport;
 import org.apache.activemq.transport.TransportFactory;
-import org.apache.activemq.transport.TransportLogger;
+import org.apache.activemq.transport.TransportLoggerFactory;
 import org.apache.activemq.transport.TransportServer;
 import org.apache.activemq.transport.WireFormatNegotiator;
 import org.apache.activemq.util.IOExceptionSupport;
@@ -40,6 +40,10 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+/**
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com (logging improvement modifications)
+ * @version $Revision$
+ */
 public class TcpTransportFactory extends TransportFactory {
     private static final Log LOG = LogFactory.getLog(TcpTransportFactory.class);
 
@@ -84,7 +88,12 @@
         tcpTransport.setSocketOptions(socketOptions);
         
         if (tcpTransport.isTrace()) {
-            transport = new TransportLogger(transport);
+            try {
+                transport = TransportLoggerFactory.getInstance().createTransportLogger(transport, tcpTransport.getLogWriterName(),
+                        tcpTransport.isDynamicManagement(), tcpTransport.isStartLogging(), tcpTransport.getJmxPort());
+            } catch (Throwable e) {
+                LOG.error("Could not create TransportLogger object for: " + tcpTransport.getLogWriterName() + ", reason: " + e, e);
+            }
         }
 
         if (isUseInactivityMonitor(transport)) {

Modified: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransportServer.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransportServer.java?rev=612661&r1=612660&r2=612661&view=diff
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransportServer.java (original)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/tcp/TcpTransportServer.java Wed Jan 16 16:48:02 2008
@@ -37,6 +37,7 @@
 import org.apache.activemq.command.BrokerInfo;
 import org.apache.activemq.openwire.OpenWireFormatFactory;
 import org.apache.activemq.transport.Transport;
+import org.apache.activemq.transport.TransportLoggerFactory;
 import org.apache.activemq.transport.TransportServer;
 import org.apache.activemq.transport.TransportServerThreadSupport;
 import org.apache.activemq.util.IOExceptionSupport;
@@ -49,6 +50,7 @@
 /**
  * A TCP based implementation of {@link TransportServer}
  * 
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com (logging improvement modifications)
  * @version $Revision: 1.1 $
  */
 
@@ -72,6 +74,12 @@
      */
     protected boolean trace = false;
     /**
+     * Name of the LogWriter implementation to use.
+     * Names are mapped to classes in the resources/META-INF/services/org/apache/activemq/transport/logwriters directory.
+     * This parameter is most probably set in Connection or TransportConnector URIs.
+     */
+    protected String logWriterName = TransportLoggerFactory.defaultLogWriterName;
+    /**
      * Specifies if the TransportLogger will be manageable by JMX or not.
      * Also, as long as there is at least 1 TransportLogger which is manageable,
      * a TransportLoggerControl MBean will me created.
@@ -177,6 +185,14 @@
     public void setTrace(boolean trace) {
         this.trace = trace;
     }
+    
+    public String getLogWriterName() {
+        return logWriterName;
+    }
+
+    public void setLogWriterName(String logFormat) {
+        this.logWriterName = logFormat;
+    }        
 
     public boolean isDynamicManagement() {
         return dynamicManagement;
@@ -310,6 +326,7 @@
             options.put("minmumWireFormatVersion", Integer
                     .valueOf(minmumWireFormatVersion));
             options.put("trace", Boolean.valueOf(trace));
+            options.put("logWriterName", logWriterName);
             options
                     .put("dynamicManagement", Boolean
                             .valueOf(dynamicManagement));

Modified: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/udp/UdpTransportFactory.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/udp/UdpTransportFactory.java?rev=612661&r1=612660&r2=612661&view=diff
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/udp/UdpTransportFactory.java (original)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/transport/udp/UdpTransportFactory.java Wed Jan 16 16:48:02 2008
@@ -28,7 +28,7 @@
 import org.apache.activemq.transport.InactivityMonitor;
 import org.apache.activemq.transport.Transport;
 import org.apache.activemq.transport.TransportFactory;
-import org.apache.activemq.transport.TransportLogger;
+import org.apache.activemq.transport.TransportLoggerFactory;
 import org.apache.activemq.transport.TransportServer;
 import org.apache.activemq.transport.reliable.DefaultReplayStrategy;
 import org.apache.activemq.transport.reliable.ExceptionIfDroppedReplayStrategy;
@@ -43,8 +43,14 @@
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 
+/**
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com (logging improvement modifications)
+ * @version $Revision$
+ */
 public class UdpTransportFactory extends TransportFactory {
+    
     private static final Log log = LogFactory.getLog(TcpTransportFactory.class);
+    
     public TransportServer doBind(String brokerId, final URI location) throws IOException {
         try {
             Map<String, String> options = new HashMap<String, String>(URISupport.parseParamters(location));
@@ -78,7 +84,11 @@
         transport = new CommandJoiner(transport, asOpenWireFormat(format));
 
         if (udpTransport.isTrace()) {
-            transport = new TransportLogger(transport);
+            try {
+                transport = TransportLoggerFactory.getInstance().createTransportLogger(transport);
+            } catch (Throwable e) {
+                log.error("Could not create TransportLogger object for: " + TransportLoggerFactory.defaultLogWriterName + ", reason: " + e, e);
+            }
         }
 
         transport = new InactivityMonitor(transport);
@@ -110,7 +120,7 @@
         OpenWireFormat openWireFormat = asOpenWireFormat(format);
 
         if (udpTransport.isTrace()) {
-            transport = new TransportLogger(transport);
+            transport = TransportLoggerFactory.getInstance().createTransportLogger(transport);
         }
 
         transport = new InactivityMonitor(transport);

Added: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/util/LogWriterFinder.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/java/org/apache/activemq/util/LogWriterFinder.java?rev=612661&view=auto
==============================================================================
--- activemq/trunk/activemq-core/src/main/java/org/apache/activemq/util/LogWriterFinder.java (added)
+++ activemq/trunk/activemq-core/src/main/java/org/apache/activemq/util/LogWriterFinder.java Wed Jan 16 16:48:02 2008
@@ -0,0 +1,141 @@
+/**
+ * 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.util;
+
+import java.io.BufferedInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Properties;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.activemq.transport.LogWriter;
+import org.apache.activemq.transport.TransportLoggerView;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * Class used to find a LogWriter implementation, and returning
+ * a LogWriter object, taking as argument the name of a log writer.
+ * The mapping between the log writer names and the classes
+ * implementing LogWriter is specified by the files in the
+ * resources/META-INF/services/org/apache/activemq/transport/logwriters
+ * directory.
+ * 
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com
+ * @version $Revision$
+ */
+public class LogWriterFinder {
+    
+    private static final Log log = LogFactory.getLog(TransportLoggerView.class);
+
+    private final String path;
+    private final ConcurrentHashMap classMap = new ConcurrentHashMap();
+
+    /**
+     * Builds a LogWriterFinder that will look for the mappings between
+     * LogWriter names and classes in the directory "path".
+     * @param path The directory where the files that map log writer names to
+     * LogWriter classes are. 
+     */
+    public LogWriterFinder(String path) {
+        this.path = path;
+    }
+
+    /**
+     * Returns a LogWriter object, given a log writer name (for example "default", or "custom").
+     * Uses a ConcurrentHashMap to cache the Class objects that have already been loaded.
+     * @param logWriterName a log writer name (for example "default", or "custom").
+     * @return a LogWriter object to be used by the TransportLogger class.
+     * @throws IllegalAccessException
+     * @throws InstantiationException
+     * @throws IOException
+     * @throws ClassNotFoundException
+     */
+    public LogWriter newInstance(String logWriterName)
+    throws IllegalAccessException, InstantiationException, IOException, ClassNotFoundException
+    {
+        Class clazz = (Class) classMap.get(logWriterName);
+        if (clazz == null) {
+            clazz = newInstance(doFindLogWriterProperties(logWriterName));
+            classMap.put(logWriterName, clazz);
+        }
+        return (LogWriter)clazz.newInstance();
+    }
+    
+    /**
+     * Loads and returns a class given a Properties object with a "class" property.
+     * @param properties a Properties object with a "class" property.
+     * @return a Class object.
+     * @throws ClassNotFoundException
+     * @throws IOException
+     */
+    private Class newInstance(Properties properties) throws ClassNotFoundException, IOException {
+
+        String className = properties.getProperty("class");
+        if (className == null) {
+            throw new IOException("Expected property is missing: " + "class");
+        }
+        Class clazz;
+        try {
+            clazz = Thread.currentThread().getContextClassLoader().loadClass(className);
+        } catch (ClassNotFoundException e) {
+            clazz = LogWriterFinder.class.getClassLoader().loadClass(className);
+        }
+
+        return clazz;
+    }
+
+    /**
+     * Given a log writer name, returns a Properties object with a "class" property
+     * whose value is a String with the name of the class to be loaded.
+     * @param logWriterName a log writer name.
+     * @return a Properties object with a "class" property
+     * @throws IOException
+     */
+    protected Properties doFindLogWriterProperties (String logWriterName) throws IOException {
+
+        String uri = path + logWriterName;
+
+        // lets try the thread context class loader first
+        ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+        if (classLoader == null) classLoader = getClass().getClassLoader();
+        InputStream in = classLoader.getResourceAsStream(uri);
+        if (in == null) {
+            in = LogWriterFinder.class.getClassLoader().getResourceAsStream(uri);
+            if (in == null) {
+                log.error("Could not find log writer for resource: " + uri);
+                throw new IOException("Could not find log writer for resource: " + uri);
+            }
+        }
+
+        // lets load the file
+        BufferedInputStream reader = null;
+        Properties properties = new Properties();
+        try {
+            reader = new BufferedInputStream(in);
+            properties.load(reader);
+            return properties;
+        } finally {
+            try {
+                reader.close();
+            } catch (Exception e) {
+            }
+        }
+    }
+
+
+}

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/util/LogWriterFinder.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: activemq/trunk/activemq-core/src/main/java/org/apache/activemq/util/LogWriterFinder.java
------------------------------------------------------------------------------
    svn:keywords = Rev Date

Added: activemq/trunk/activemq-core/src/main/resources/META-INF/services/org/apache/activemq/transport/logwriters/custom
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/resources/META-INF/services/org/apache/activemq/transport/logwriters/custom?rev=612661&view=auto
==============================================================================
--- activemq/trunk/activemq-core/src/main/resources/META-INF/services/org/apache/activemq/transport/logwriters/custom (added)
+++ activemq/trunk/activemq-core/src/main/resources/META-INF/services/org/apache/activemq/transport/logwriters/custom Wed Jan 16 16:48:02 2008
@@ -0,0 +1,17 @@
+## ---------------------------------------------------------------------------
+## 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.
+## ---------------------------------------------------------------------------
+class=org.apache.activemq.transport.logwriters.CustomLogWriter

Added: activemq/trunk/activemq-core/src/main/resources/META-INF/services/org/apache/activemq/transport/logwriters/default
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-core/src/main/resources/META-INF/services/org/apache/activemq/transport/logwriters/default?rev=612661&view=auto
==============================================================================
--- activemq/trunk/activemq-core/src/main/resources/META-INF/services/org/apache/activemq/transport/logwriters/default (added)
+++ activemq/trunk/activemq-core/src/main/resources/META-INF/services/org/apache/activemq/transport/logwriters/default Wed Jan 16 16:48:02 2008
@@ -0,0 +1,17 @@
+## ---------------------------------------------------------------------------
+## 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.
+## ---------------------------------------------------------------------------
+class=org.apache.activemq.transport.logwriters.DefaultLogWriter

Modified: activemq/trunk/activemq-optional/src/main/java/org/apache/activemq/transport/http/HttpTransportFactory.java
URL: http://svn.apache.org/viewvc/activemq/trunk/activemq-optional/src/main/java/org/apache/activemq/transport/http/HttpTransportFactory.java?rev=612661&r1=612660&r2=612661&view=diff
==============================================================================
--- activemq/trunk/activemq-optional/src/main/java/org/apache/activemq/transport/http/HttpTransportFactory.java (original)
+++ activemq/trunk/activemq-optional/src/main/java/org/apache/activemq/transport/http/HttpTransportFactory.java Wed Jan 16 16:48:02 2008
@@ -22,7 +22,7 @@
 
 import org.apache.activemq.transport.Transport;
 import org.apache.activemq.transport.TransportFactory;
-import org.apache.activemq.transport.TransportLogger;
+import org.apache.activemq.transport.TransportLoggerFactory;
 import org.apache.activemq.transport.TransportServer;
 import org.apache.activemq.transport.util.TextWireFormat;
 import org.apache.activemq.transport.xstream.XStreamWireFormat;
@@ -31,6 +31,7 @@
 import org.apache.commons.logging.LogFactory;
 
 /**
+ * @author David Martin Clavo david(dot)martin(dot)clavo(at)gmail.com (logging improvement modifications)
  * @version $Revision$
  */
 public class HttpTransportFactory extends TransportFactory {
@@ -59,10 +60,14 @@
     }
 
     public Transport compositeConfigure(Transport transport, WireFormat format, Map options) {
-        HttpClientTransport httpTransport = (HttpClientTransport)super.compositeConfigure(transport, format, options);
+        HttpClientTransport httpTransport = (HttpClientTransport) super.compositeConfigure(transport, format, options);
         transport = httpTransport;
-        if (httpTransport.isTrace()) {
-            transport = new TransportLogger(httpTransport);
+        if( httpTransport.isTrace() ) {
+            try {
+                transport = TransportLoggerFactory.getInstance().createTransportLogger(transport);
+            } catch (Throwable e) {
+                LOG.error("Could not create TransportLogger object for: " + TransportLoggerFactory.defaultLogWriterName + ", reason: " + e, e);
+            }
         }
         return transport;
     }