Repository: logging-log4j2
Updated Branches:
  refs/heads/master d8ede120f -> 0812a6795


LOG4J2-323 Better web app support for async loggers (when config file
contains AsyncRoot/AsyncLogger).

Fixed a memory leak that occurred when the logging jars are placed in
the container's classpath and the configuration file uses
AsyncRoot/AsyncLogger.
The problem was that the first web application started the Disruptor
background thread [AsyncLoggerConfig-1] but did not stop it until all
web apps are stopped.
Each web application now has its own Disruptor which is stopped/started
together with the web app.

- AsyncLoggerConfig does not create Helper objects, but instead obtain
shared instances from the Configuration
- Added method getAsyncLoggerConfigDelegate() to Configuration
- AsyncLoggerConfigHelper disruptor is no longer a static field:
multiple Helper instances each have their own Disruptor. Start/stop no
longer need a reference count.
- AbstractConfiguration simplified stop() impl: simply call
asyncLoggerConfigHelper.stop(), no need to worry about ref counts (order
in which components are stopped still matters)
- Fixed tests


Project: http://git-wip-us.apache.org/repos/asf/logging-log4j2/repo
Commit: http://git-wip-us.apache.org/repos/asf/logging-log4j2/commit/0812a679
Tree: http://git-wip-us.apache.org/repos/asf/logging-log4j2/tree/0812a679
Diff: http://git-wip-us.apache.org/repos/asf/logging-log4j2/diff/0812a679

Branch: refs/heads/master
Commit: 0812a6795743015a9c49440f9d262f378d095186
Parents: d8ede12
Author: rpopma <rpo...@apache.org>
Authored: Sat Oct 24 01:37:36 2015 +0900
Committer: rpopma <rpo...@apache.org>
Committed: Sat Oct 24 01:37:36 2015 +0900

----------------------------------------------------------------------
 .../log4j/core/async/AsyncLoggerConfig.java     |   9 +-
 .../core/async/AsyncLoggerConfigDelegate.java   |   2 +-
 .../core/async/AsyncLoggerConfigHelper.java     | 171 +++++++++----------
 .../core/config/AbstractConfiguration.java      | 124 +++++++-------
 .../log4j/core/config/Configuration.java        |  15 +-
 .../core/async/AsyncLoggerLocationTest.java     |   6 +-
 .../log4j/core/async/AsyncLoggerTest.java       |   2 +-
 .../async/AsyncLoggerThreadContextTest.java     |   2 +-
 .../async/AsyncLoggerTimestampMessageTest.java  |   2 +-
 src/changes/changes.xml                         |   6 +
 10 files changed, 174 insertions(+), 165 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/0812a679/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfig.java
----------------------------------------------------------------------
diff --git 
a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfig.java
 
b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfig.java
index 7be1e8f..b109f49 100644
--- 
a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfig.java
+++ 
b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfig.java
@@ -79,6 +79,7 @@ public class AsyncLoggerConfig extends LoggerConfig {
             final boolean includeLocation) {
         super(name, appenders, filter, level, additive, properties, config,
                 includeLocation);
+        helper = config.getAsyncLoggerConfigDelegate();
     }
 
     /**
@@ -109,20 +110,12 @@ public class AsyncLoggerConfig extends LoggerConfig {
     @Override
     public void start() {
         LOGGER.trace("AsyncLoggerConfig[{}] starting...", displayName());
-        this.setStarting();
-        if (helper == null) {
-            helper = new AsyncLoggerConfigHelper();
-        } else {
-            AsyncLoggerConfigHelper.claim(); // LOG4J2-336
-        }
         super.start();
     }
 
     @Override
     public void stop() {
         LOGGER.trace("AsyncLoggerConfig[{}] stopping...", displayName());
-        this.setStopping();
-        AsyncLoggerConfigHelper.release();
         super.stop();
     }
 

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/0812a679/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDelegate.java
----------------------------------------------------------------------
diff --git 
a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDelegate.java
 
b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDelegate.java
index 229b08e..304626f 100644
--- 
a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDelegate.java
+++ 
b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDelegate.java
@@ -24,7 +24,7 @@ import org.apache.logging.log4j.core.jmx.RingBufferAdmin;
  * Encapsulates the mechanism used to log asynchronously. There is one 
delegate per configuration, which is shared by
  * all AsyncLoggerConfig objects in the configuration.
  */
-interface AsyncLoggerConfigDelegate {
+public interface AsyncLoggerConfigDelegate {
 
     /**
      * If possible, delegates the invocation of {@code callAppenders} to the 
background thread and returns {@code true}.

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/0812a679/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigHelper.java
----------------------------------------------------------------------
diff --git 
a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigHelper.java
 
b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigHelper.java
index 1efdd18..a8cf158 100644
--- 
a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigHelper.java
+++ 
b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigHelper.java
@@ -29,7 +29,6 @@ import org.apache.logging.log4j.util.PropertiesUtil;
 
 import com.lmax.disruptor.BlockingWaitStrategy;
 import com.lmax.disruptor.EventFactory;
-import com.lmax.disruptor.EventHandler;
 import com.lmax.disruptor.EventTranslatorTwoArg;
 import com.lmax.disruptor.ExceptionHandler;
 import com.lmax.disruptor.RingBuffer;
@@ -53,7 +52,7 @@ import com.lmax.disruptor.dsl.ProducerType;
  * This class serves to make the dependency on the Disruptor optional, so that 
these classes are only loaded when the
  * {@code AsyncLoggerConfig} is actually used.
  */
-class AsyncLoggerConfigHelper implements AsyncLoggerConfigDelegate {
+public class AsyncLoggerConfigHelper implements AsyncLoggerConfigDelegate {
 
     private static final int MAX_DRAIN_ATTEMPTS_BEFORE_SHUTDOWN = 200;
     private static final int SLEEP_MILLIS_BETWEEN_DRAIN_ATTEMPTS = 50;
@@ -61,12 +60,56 @@ class AsyncLoggerConfigHelper implements 
AsyncLoggerConfigDelegate {
     private static final int RINGBUFFER_DEFAULT_SIZE = 256 * 1024;
     private static final Logger LOGGER = StatusLogger.getLogger();
 
-    private static ThreadFactory threadFactory = new 
DaemonThreadFactory("AsyncLoggerConfig-");
-    private static volatile Disruptor<Log4jEventWrapper> disruptor;
-    private static ExecutorService executor;
+    /**
+     * RingBuffer events contain all information necessary to perform the work 
in a separate thread.
+     */
+    private static class Log4jEventWrapper {
+        private AsyncLoggerConfig loggerConfig;
+        private LogEvent event;
+
+        /**
+         * Release references held by ring buffer to allow objects to be 
garbage-collected.
+         */
+        public void clear() {
+            loggerConfig = null;
+            event = null;
+        }
+    }
+
+    /**
+     * EventHandler performs the work in a separate thread.
+     */
+    private static class Log4jEventWrapperHandler implements 
SequenceReportingEventHandler<Log4jEventWrapper> {
+        private static final int NOTIFY_PROGRESS_THRESHOLD = 50;
+        private Sequence sequenceCallback;
+        private int counter;
+
+        @Override
+        public void setSequenceCallback(final Sequence sequenceCallback) {
+            this.sequenceCallback = sequenceCallback;
+        }
+
+        @Override
+        public void onEvent(final Log4jEventWrapper event, final long 
sequence, final boolean endOfBatch)
+                throws Exception {
+            event.event.setEndOfBatch(endOfBatch);
+            event.loggerConfig.asyncCallAppenders(event.event);
+            event.clear();
 
-    private static volatile int count = 0;
-    private static ThreadLocal<Boolean> isAppenderThread = new ThreadLocal<>();
+            notifyIntermediateProgress(sequence);
+        }
+
+        /**
+         * Notify the BatchEventProcessor that the sequence has progressed. 
Without this callback the sequence would not
+         * be progressed until the batch has completely finished.
+         */
+        private void notifyIntermediateProgress(final long sequence) {
+            if (++counter > NOTIFY_PROGRESS_THRESHOLD) {
+                sequenceCallback.set(sequence);
+                counter = 0;
+            }
+        }
+    }
 
     /**
      * Factory used to populate the RingBuffer with events. These event 
objects are then re-used during the life of the
@@ -82,7 +125,7 @@ class AsyncLoggerConfigHelper implements 
AsyncLoggerConfigDelegate {
     /**
      * Object responsible for passing on data to a specific RingBuffer event.
      */
-    private final EventTranslatorTwoArg<Log4jEventWrapper, LogEvent, 
AsyncLoggerConfig> translator =
+    private final static EventTranslatorTwoArg<Log4jEventWrapper, LogEvent, 
AsyncLoggerConfig> translator =
             new EventTranslatorTwoArg<Log4jEventWrapper, LogEvent, 
AsyncLoggerConfig>() {
 
         @Override
@@ -93,30 +136,42 @@ class AsyncLoggerConfigHelper implements 
AsyncLoggerConfigDelegate {
         }
     };
 
+    private static ThreadFactory threadFactory = new 
DaemonThreadFactory("AsyncLoggerConfig-");
+    private static final ThreadLocal<Boolean> isAppenderThread = new 
ThreadLocal<>();
+
+    private volatile Disruptor<Log4jEventWrapper> disruptor;
+    private ExecutorService executor;
+
     public AsyncLoggerConfigHelper() {
-        claim();
     }
 
-    private static synchronized void initDisruptor() {
+    /**
+     * Increases the reference count and creates and starts a new Disruptor 
and associated thread if none currently
+     * exists.
+     * 
+     * @see #release()
+     */
+    public synchronized void start() {
         if (disruptor != null) {
-            LOGGER.trace("AsyncLoggerConfigHelper not starting new disruptor, 
using existing object. Ref count is {}.",
-                    count);
+            LOGGER.trace("AsyncLoggerConfigHelper not starting new disruptor 
for this configuration, using existing object.");
             return;
         }
-        LOGGER.trace("AsyncLoggerConfigHelper creating new disruptor. Ref 
count is {}.", count);
+        LOGGER.trace("AsyncLoggerConfigHelper creating new disruptor for this 
configuration.");
         final int ringBufferSize = calculateRingBufferSize();
         final WaitStrategy waitStrategy = createWaitStrategy();
         executor = Executors.newSingleThreadExecutor(threadFactory);
-        initThreadLocalForExecutorThread();
+        initThreadLocalForExecutorThread(executor);
+        
         disruptor = new Disruptor<>(FACTORY, ringBufferSize, executor, 
ProducerType.MULTI, waitStrategy);
-        final EventHandler<Log4jEventWrapper>[] handlers = new 
Log4jEventWrapperHandler[] {//
-        new Log4jEventWrapperHandler()};
+
         final ExceptionHandler<Log4jEventWrapper> errorHandler = 
getExceptionHandler();
         disruptor.handleExceptionsWith(errorHandler);
+
+        final Log4jEventWrapperHandler[] handlers = {new 
Log4jEventWrapperHandler()};
         disruptor.handleEventsWith(handlers);
 
         LOGGER.debug(
-                "Starting AsyncLoggerConfig disruptor with ringbufferSize={}, 
waitStrategy={}, exceptionHandler={}...",
+                "Starting AsyncLoggerConfig disruptor for this configuration 
with ringbufferSize={}, waitStrategy={}, exceptionHandler={}...",
                 disruptor.getRingBuffer().getBufferSize(), 
waitStrategy.getClass().getSimpleName(), errorHandler);
         disruptor.start();
     }
@@ -171,87 +226,18 @@ class AsyncLoggerConfigHelper implements 
AsyncLoggerConfigDelegate {
     }
 
     /**
-     * RingBuffer events contain all information necessary to perform the work 
in a separate thread.
-     */
-    private static class Log4jEventWrapper {
-        private AsyncLoggerConfig loggerConfig;
-        private LogEvent event;
-
-        /**
-         * Release references held by ring buffer to allow objects to be 
garbage-collected.
-         */
-        public void clear() {
-            loggerConfig = null;
-            event = null;
-        }
-    }
-
-    /**
-     * EventHandler performs the work in a separate thread.
-     */
-    private static class Log4jEventWrapperHandler implements 
SequenceReportingEventHandler<Log4jEventWrapper> {
-        private static final int NOTIFY_PROGRESS_THRESHOLD = 50;
-        private Sequence sequenceCallback;
-        private int counter;
-
-        @Override
-        public void setSequenceCallback(final Sequence sequenceCallback) {
-            this.sequenceCallback = sequenceCallback;
-        }
-
-        @Override
-        public void onEvent(final Log4jEventWrapper event, final long 
sequence, final boolean endOfBatch)
-                throws Exception {
-            event.event.setEndOfBatch(endOfBatch);
-            event.loggerConfig.asyncCallAppenders(event.event);
-            event.clear();
-
-            notifyIntermediateProgress(sequence);
-        }
-
-        /**
-         * Notify the BatchEventProcessor that the sequence has progressed. 
Without this callback the sequence would not
-         * be progressed until the batch has completely finished.
-         */
-        private void notifyIntermediateProgress(final long sequence) {
-            if (++counter > NOTIFY_PROGRESS_THRESHOLD) {
-                sequenceCallback.set(sequence);
-                counter = 0;
-            }
-        }
-    }
-
-    /**
-     * Increases the reference count and creates and starts a new Disruptor 
and associated thread if none currently
-     * exists.
-     * 
-     * @see #release()
-     */
-    static synchronized void claim() {
-        count++;
-        initDisruptor();
-    }
-
-    /**
      * Decreases the reference count. If the reference count reached zero, the 
Disruptor and its associated thread are
      * shut down and their references set to {@code null}.
      */
-    static synchronized void release() {
-        if (--count > 0) {
-            LOGGER.trace("AsyncLoggerConfigHelper: not shutting down 
disruptor: ref count is {}.", count);
-            return;
-        }
+    public synchronized void stop() {
         final Disruptor<Log4jEventWrapper> temp = disruptor;
         if (temp == null) {
-            LOGGER.trace("AsyncLoggerConfigHelper: disruptor already shut 
down: ref count is {}. (Resetting to zero.)",
-                    count);
-            count = 0; // ref count must not be negative or #claim() will not 
work correctly
+            LOGGER.trace("AsyncLoggerConfigHelper: disruptor for this 
configuration already shut down.");
             return; // disruptor was already shut down by another thread
         }
-        LOGGER.trace("AsyncLoggerConfigHelper: shutting down disruptor: ref 
count is {}.", count);
+        LOGGER.trace("AsyncLoggerConfigHelper: shutting down disruptor for 
this configuration.");
 
-        // Must guarantee that publishing to the RingBuffer has stopped
-        // before we call disruptor.shutdown()
+        // We must guarantee that publishing to the RingBuffer has stopped 
before we call disruptor.shutdown().
         disruptor = null; // client code fails with NPE if log after stop = OK
 
         // Calling Disruptor.shutdown() will wait until all enqueued events 
are fully processed,
@@ -264,6 +250,8 @@ class AsyncLoggerConfigHelper implements 
AsyncLoggerConfigDelegate {
             }
         }
         temp.shutdown(); // busy-spins until all events currently in the 
disruptor have been processed
+
+        LOGGER.trace("AsyncLoggerConfigHelper: shutting down disruptor 
executor for this configuration.");
         executor.shutdown(); // finally, kill the processor thread
         executor = null; // release reference to allow GC
     }
@@ -279,8 +267,9 @@ class AsyncLoggerConfigHelper implements 
AsyncLoggerConfigDelegate {
     /**
      * Initialize the threadlocal that allows us to detect Logger.log() calls 
initiated from the appender thread, which
      * may cause deadlock when the RingBuffer is full. (LOG4J2-471)
+     * @param executor contains the appender background thread
      */
-    private static void initThreadLocalForExecutorThread() {
+    private static void initThreadLocalForExecutorThread(final ExecutorService 
executor) {
         executor.submit(new Runnable() {
             @Override
             public void run() {

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/0812a679/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AbstractConfiguration.java
----------------------------------------------------------------------
diff --git 
a/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AbstractConfiguration.java
 
b/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AbstractConfiguration.java
index 679e066..3164d96 100644
--- 
a/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AbstractConfiguration.java
+++ 
b/log4j-core/src/main/java/org/apache/logging/log4j/core/config/AbstractConfiguration.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
@@ -36,7 +37,6 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.ScheduledThreadPoolExecutor;
 
 import org.apache.logging.log4j.Level;
-import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.core.Appender;
 import org.apache.logging.log4j.core.Filter;
 import org.apache.logging.log4j.core.Layout;
@@ -44,13 +44,13 @@ import org.apache.logging.log4j.core.LogEvent;
 import org.apache.logging.log4j.core.appender.AsyncAppender;
 import org.apache.logging.log4j.core.appender.ConsoleAppender;
 import org.apache.logging.log4j.core.async.AsyncLoggerConfig;
-import org.apache.logging.log4j.core.async.AsyncLoggerContextSelector;
+import org.apache.logging.log4j.core.async.AsyncLoggerConfigDelegate;
+import org.apache.logging.log4j.core.async.AsyncLoggerConfigHelper;
 import org.apache.logging.log4j.core.async.DaemonThreadFactory;
 import org.apache.logging.log4j.core.config.plugins.util.PluginBuilder;
 import org.apache.logging.log4j.core.config.plugins.util.PluginManager;
 import org.apache.logging.log4j.core.config.plugins.util.PluginType;
 import org.apache.logging.log4j.core.filter.AbstractFilterable;
-import org.apache.logging.log4j.core.impl.Log4jContextFactory;
 import org.apache.logging.log4j.core.layout.PatternLayout;
 import org.apache.logging.log4j.core.lookup.Interpolator;
 import org.apache.logging.log4j.core.lookup.MapLookup;
@@ -60,12 +60,10 @@ import org.apache.logging.log4j.core.net.Advertiser;
 import org.apache.logging.log4j.core.script.AbstractScript;
 import org.apache.logging.log4j.core.script.ScriptManager;
 import org.apache.logging.log4j.core.script.ScriptRef;
-import org.apache.logging.log4j.core.selector.ContextSelector;
 import org.apache.logging.log4j.core.util.Constants;
 import org.apache.logging.log4j.core.util.Loader;
 import org.apache.logging.log4j.core.util.NameUtil;
 import org.apache.logging.log4j.core.util.WatchManager;
-import org.apache.logging.log4j.spi.LoggerContextFactory;
 import org.apache.logging.log4j.util.PropertiesUtil;
 
 /**
@@ -126,6 +124,7 @@ public abstract class AbstractConfiguration extends 
AbstractFilterable implement
     private ScriptManager scriptManager;
     private ScheduledExecutorService executorService;
     private final WatchManager watchManager = new WatchManager();
+    private AsyncLoggerConfigHelper asyncLoggerConfigHelper;
 
     /**
      * Constructor.
@@ -167,6 +166,16 @@ public abstract class AbstractConfiguration extends 
AbstractFilterable implement
         return executorService;
     }
 
+       @Override
+       public AsyncLoggerConfigDelegate getAsyncLoggerConfigDelegate() {
+           // lazily instantiate only when requested by AsyncLoggers:
+           // loading AsyncLoggerConfigHelper requires LMAX Disruptor jar on 
classpath
+           if (asyncLoggerConfigHelper == null) {
+               asyncLoggerConfigHelper = new AsyncLoggerConfigHelper();
+           }
+               return asyncLoggerConfigHelper;
+       }
+
     /**
      * Initialize the configuration.
      */
@@ -215,6 +224,9 @@ public abstract class AbstractConfiguration extends 
AbstractFilterable implement
         if (watchManager.getIntervalSeconds() > 0) {
             watchManager.start();
         }
+        if (hasAsyncLoggers()) {
+               asyncLoggerConfigHelper.start();
+        }
         final Set<LoggerConfig> alreadyStarted = new HashSet<>();
         for (final LoggerConfig logger : loggers.values()) {
             logger.start();
@@ -230,7 +242,19 @@ public abstract class AbstractConfiguration extends 
AbstractFilterable implement
         LOGGER.debug("Started configuration {} OK.", this);
     }
 
-    /**
+    private boolean hasAsyncLoggers() {
+        if (root instanceof AsyncLoggerConfig) {
+            return true;
+        }
+        for (final LoggerConfig logger : loggers.values()) {
+            if (logger instanceof AsyncLoggerConfig) {
+               return true;
+            }
+        }
+               return false;
+       }
+
+       /**
      * Tear down the configuration.
      */
     @Override
@@ -238,47 +262,39 @@ public abstract class AbstractConfiguration extends 
AbstractFilterable implement
         this.setStopping();
         LOGGER.trace("Stopping {}...", this);
 
+        // Stop the components that are closest to the application first:
+        // 1. Notify all LoggerConfigs' ReliabilityStrategy that the 
configuration will be stopped.
+        // 2. Stop the LoggerConfig objects (this may stop nested Filters)
+        // 3. Stop the AsyncLoggerConfigDelegate. This shuts down the 
AsyncLoggerConfig Disruptor
+        //    and waits until all events in the RingBuffer have been processed.
+        // 4. Stop all AsyncAppenders. This shuts down the associated thread 
and
+        //    waits until all events in the queue have been processed. (With 
optional timeout.)
+        // 5. Notify all LoggerConfigs' ReliabilityStrategy that appenders 
will be stopped.
+        //    This guarantees that any event received by a LoggerConfig before 
reconfiguration
+        //    are passed on to the Appenders before the Appenders are stopped.
+        // 6. Stop the remaining running Appenders. (It should now be safe to 
do so.)
+        // 7. Notify all LoggerConfigs that their Appenders can be cleaned up.
+
         for (final LoggerConfig loggerConfig : loggers.values()) {
             
loggerConfig.getReliabilityStrategy().beforeStopConfiguration(this);
         }
         final String cls = getClass().getSimpleName();
         LOGGER.trace("{} notified {} ReliabilityStrategies that config will be 
stopped.", cls, loggers.size());
-
-        // LOG4J2-392 first stop AsyncLogger Disruptor thread
-        final LoggerContextFactory factory = LogManager.getFactory();
-        if (factory instanceof Log4jContextFactory) {
-            final ContextSelector selector = ((Log4jContextFactory) 
factory).getSelector();
-            if (selector instanceof AsyncLoggerContextSelector) { // all 
loggers are async
-                // TODO until LOG4J2-493 is fixed we can only stop AsyncLogger 
once!
-                // but LoggerContext.setConfiguration will call config.stop()
-                // every time the configuration changes...
-                //
-                // Uncomment the line below after LOG4J2-493 is fixed
-                // AsyncLogger.stop();
-                // LOGGER.trace("AbstractConfiguration stopped AsyncLogger 
disruptor.");
-            }
-        }
-        // similarly, first stop AsyncLoggerConfig Disruptor thread(s)
-        final Set<LoggerConfig> alreadyStopped = new HashSet<>();
-        int asyncLoggerConfigCount = 0;
+        
+        LOGGER.trace("{} stopping {} LoggerConfigs.", cls, loggers.size());
         for (final LoggerConfig logger : loggers.values()) {
-            if (logger instanceof AsyncLoggerConfig) {
-                // LOG4J2-520, LOG4J2-392:
-                // Important: do not clear appenders until after all 
AsyncLoggerConfigs
-                // have been stopped! Stopping the last AsyncLoggerConfig will
-                // shut down the disruptor and wait for all enqueued events to 
be processed.
-                // Only *after this* the appenders can be cleared or events 
will be lost.
-                logger.stop();
-                asyncLoggerConfigCount++;
-                alreadyStopped.add(logger);
-            }
+            logger.stop();
         }
-        if (root instanceof AsyncLoggerConfig & 
!alreadyStopped.contains(root)) { // LOG4J2-807
+        if (!root.isStopped()) {
             root.stop();
-            asyncLoggerConfigCount++;
-            alreadyStopped.add(root);
         }
-        LOGGER.trace("{} stopped {} AsyncLoggerConfigs.", cls, 
asyncLoggerConfigCount);
+
+        if (hasAsyncLoggers()) {
+            LOGGER.trace("{} stopping AsyncLoggerConfigDelegate.", cls);
+            asyncLoggerConfigHelper.stop();
+        }
+        
+        LOGGER.trace("{} stopping AsyncAppenders.", cls);
 
         // Stop the appenders in reverse order in case they still have 
activity.
         final Appender[] array = appenders.values().toArray(new 
Appender[appenders.size()]);
@@ -293,11 +309,12 @@ public abstract class AbstractConfiguration extends 
AbstractFilterable implement
         }
         LOGGER.trace("{} stopped {} AsyncAppenders.", cls, asyncAppenderCount);
 
+        LOGGER.trace("{} notifying ReliabilityStrategies that appenders will 
be stopped.", cls);
         for (final LoggerConfig loggerConfig : loggers.values()) {
             loggerConfig.getReliabilityStrategy().beforeStopAppenders();
         }
-        LOGGER.trace("{} notified {} ReliabilityStrategies that appenders will 
be stopped.", cls, loggers.size());
 
+        LOGGER.trace("{} stopping remaining Appenders.", cls);
         int appenderCount = 0;
         for (int i = array.length - 1; i >= 0; --i) {
             if (array[i].isStarted()) { // then stop remaining Appenders
@@ -305,21 +322,18 @@ public abstract class AbstractConfiguration extends 
AbstractFilterable implement
                 appenderCount++;
             }
         }
-        LOGGER.trace("{} stopped {} Appenders.", cls, appenderCount);
+        LOGGER.trace("{} stopped {} remaining Appenders.", cls, appenderCount);
 
-        int loggerCount = 0;
+        LOGGER.trace("{} cleaning Appenders from {} LoggerConfigs.", cls, 
loggers.size());
         for (final LoggerConfig loggerConfig : loggers.values()) {
 
-            // AsyncLoggerConfigHelper decreases its ref count when an 
AsyncLoggerConfig is stopped.
-            // Stopping the same AsyncLoggerConfig twice results in an 
incorrect ref count and
-            // the shared Disruptor may be shut down prematurely, resulting in 
NPE or other errors.
-            if (!alreadyStopped.contains(loggerConfig)) {
-                loggerConfig.stop();
-                loggerCount++;
-            }
+            // LOG4J2-520, LOG4J2-392:
+            // Important: do not clear appenders until after all 
AsyncLoggerConfigs
+            // have been stopped! Stopping the last AsyncLoggerConfig will
+            // shut down the disruptor and wait for all enqueued events to be 
processed.
+            // Only *after this* the appenders can be cleared or events will 
be lost.
             loggerConfig.clearAppenders();
         }
-        LOGGER.trace("{} stopped {} LoggerConfigs.", cls, loggerCount);
 
         if (watchManager.isStarted()) {
             watchManager.stop();
@@ -329,12 +343,6 @@ public abstract class AbstractConfiguration extends 
AbstractFilterable implement
             executorService.shutdown();
         }
 
-        // AsyncLoggerConfigHelper decreases its ref count when an 
AsyncLoggerConfig is stopped.
-        // Stopping the same AsyncLoggerConfig twice results in an incorrect 
ref count and
-        // the shared Disruptor may be shut down prematurely, resulting in NPE 
or other errors.
-        if (!alreadyStopped.contains(root)) {
-            root.stop();
-        }
         super.stop();
         if (advertiser != null && advertisement != null) {
             advertiser.unadvertise(advertisement);
@@ -458,8 +466,10 @@ public abstract class AbstractConfiguration extends 
AbstractFilterable implement
                 copy.add(child.getObject(CustomLevelConfig.class));
                 customLevels = copy;
             } else {
-                LOGGER.error("Unknown object \"{}\" of type {} is ignored.", 
child.getName(), child.getObject()
-                        .getClass().getName());
+                final List<String> expected = Arrays.asList("\"Appenders\"", 
"\"Loggers\"", "\"Properties\"",
+                        "\"Scripts\"", "\"CustomLevels\"");
+                LOGGER.error("Unknown object \"{}\" of type {} is ignored: 
expected one of {}.", child.getName(),
+                        child.getObject().getClass().getName(), expected);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/0812a679/log4j-core/src/main/java/org/apache/logging/log4j/core/config/Configuration.java
----------------------------------------------------------------------
diff --git 
a/log4j-core/src/main/java/org/apache/logging/log4j/core/config/Configuration.java
 
b/log4j-core/src/main/java/org/apache/logging/log4j/core/config/Configuration.java
index 2bd7d1c..4a18d78 100644
--- 
a/log4j-core/src/main/java/org/apache/logging/log4j/core/config/Configuration.java
+++ 
b/log4j-core/src/main/java/org/apache/logging/log4j/core/config/Configuration.java
@@ -16,19 +16,20 @@
  */
 package org.apache.logging.log4j.core.config;
 
+import java.util.List;
+import java.util.Map;
+
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.core.Appender;
 import org.apache.logging.log4j.core.Filter;
 import org.apache.logging.log4j.core.LogEvent;
 import org.apache.logging.log4j.core.Logger;
+import org.apache.logging.log4j.core.async.AsyncLoggerConfigDelegate;
 import org.apache.logging.log4j.core.filter.Filterable;
 import org.apache.logging.log4j.core.lookup.StrSubstitutor;
 import org.apache.logging.log4j.core.net.Advertiser;
 import org.apache.logging.log4j.core.script.ScriptManager;
 
-import java.util.List;
-import java.util.Map;
-
 /**
  * Interface that must be implemented to create a configuration.
  */
@@ -147,4 +148,12 @@ public interface Configuration extends Filterable {
 
     ScriptManager getScriptManager();
 
+    /**
+     * Returns the {@code AsyncLoggerConfigDelegate} shared by all
+     * {@code AsyncLoggerConfig} instances defined in this Configuration.
+     * 
+     * @return the {@code AsyncLoggerConfigDelegate}
+     */
+       AsyncLoggerConfigDelegate getAsyncLoggerConfigDelegate();
+
 }

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/0812a679/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerLocationTest.java
----------------------------------------------------------------------
diff --git 
a/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerLocationTest.java
 
b/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerLocationTest.java
index 2b9f5ee..383bfea 100644
--- 
a/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerLocationTest.java
+++ 
b/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerLocationTest.java
@@ -36,6 +36,9 @@ public class AsyncLoggerLocationTest {
 
     @BeforeClass
     public static void beforeClass() {
+        final File file = new File("target", "AsyncLoggerLocationTest.log");
+        file.delete();
+        
         System.setProperty(Constants.LOG4J_CONTEXT_SELECTOR,
                 AsyncLoggerContextSelector.class.getName());
         System.setProperty(ConfigurationFactory.CONFIGURATION_FILE_PROPERTY,
@@ -51,11 +54,10 @@ public class AsyncLoggerLocationTest {
     public void testAsyncLogWritesToLog() throws Exception {
         final File file = new File("target", "AsyncLoggerLocationTest.log");
         // System.out.println(f.getAbsolutePath());
-        file.delete();
         final Logger log = LogManager.getLogger("com.foo.Bar");
         final String msg = "Async logger msg with location";
         log.info(msg);
-        CoreLoggerContexts.stopLoggerContext(file); // stop async thread
+        CoreLoggerContexts.stopLoggerContext(false, file); // stop async thread
 
         final BufferedReader reader = new BufferedReader(new FileReader(file));
         final String line1 = reader.readLine();

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/0812a679/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerTest.java
----------------------------------------------------------------------
diff --git 
a/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerTest.java
 
b/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerTest.java
index ad60edb..6c545d7 100644
--- 
a/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerTest.java
+++ 
b/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerTest.java
@@ -56,7 +56,7 @@ public class AsyncLoggerTest {
         final Logger log = LogManager.getLogger("com.foo.Bar");
         final String msg = "Async logger msg";
         log.info(msg, new InternalError("this is not a real error"));
-        CoreLoggerContexts.stopLoggerContext(file); // stop async thread
+        CoreLoggerContexts.stopLoggerContext(false, file); // stop async thread
 
         final BufferedReader reader = new BufferedReader(new FileReader(file));
         final String line1 = reader.readLine();

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/0812a679/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerThreadContextTest.java
----------------------------------------------------------------------
diff --git 
a/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerThreadContextTest.java
 
b/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerThreadContextTest.java
index 1e065f0..646e8b2 100644
--- 
a/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerThreadContextTest.java
+++ 
b/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerThreadContextTest.java
@@ -60,7 +60,7 @@ public class AsyncLoggerThreadContextTest {
         final Logger log = LogManager.getLogger("com.foo.Bar");
         final String msg = "Async logger msg";
         log.info(msg, new InternalError("this is not a real error"));
-        CoreLoggerContexts.stopLoggerContext(file); // stop async thread
+        CoreLoggerContexts.stopLoggerContext(false, file); // stop async thread
 
         final BufferedReader reader = new BufferedReader(new FileReader(file));
         final String line1 = reader.readLine();

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/0812a679/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerTimestampMessageTest.java
----------------------------------------------------------------------
diff --git 
a/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerTimestampMessageTest.java
 
b/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerTimestampMessageTest.java
index 829ac86..408388e 100644
--- 
a/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerTimestampMessageTest.java
+++ 
b/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerTimestampMessageTest.java
@@ -70,7 +70,7 @@ public class AsyncLoggerTimestampMessageTest {
         file.delete();
         final Logger log = LogManager.getLogger("com.foo.Bar");
         log.info(new TimeMsg("Async logger msg with embedded timestamp", 
123456789000L));
-        CoreLoggerContexts.stopLoggerContext(file); // stop async thread
+        CoreLoggerContexts.stopLoggerContext(false, file); // stop async thread
 
         final BufferedReader reader = new BufferedReader(new FileReader(file));
         final String line1 = reader.readLine();

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/0812a679/src/changes/changes.xml
----------------------------------------------------------------------
diff --git a/src/changes/changes.xml b/src/changes/changes.xml
index 44a5c77..cbdada1 100644
--- a/src/changes/changes.xml
+++ b/src/changes/changes.xml
@@ -24,6 +24,12 @@
   </properties>
   <body>
     <release version="2.5" date="2015-MM-DD" description="GA Release 2.5">
+      <action issue="LOG4J2-323" dev="rpopma" type="fix">
+        Better web app support for async loggers: Fixed a memory leak that 
occurred when the logging jars are placed
+        in the container's classpath and the configuration file uses 
AsyncRoot/AsyncLogger.
+        The problem was that the first web application started the Disruptor 
background thread [AsyncLoggerConfig-1] but did not stop it until all web apps 
are stopped.
+        Each web application now has its own Disruptor which is 
stopped/started together with the web app.
+      </action>
       <action issue="LOG4J2-493" dev="rpopma" type="fix">
         Better web app support for async loggers: it is now possible to place 
the logging jars in the container's
         classpath when making all loggers asynchronous by using 
AsyncLoggerContextSelector. This fixes a problem where

Reply via email to