You are viewing a plain text version of this content. The canonical link for it is here.
Posted to log4j-dev@logging.apache.org by "Remko Popma (JIRA)" <ji...@apache.org> on 2015/10/23 18:39:30 UTC

[jira] [Issue Comment Deleted] (LOG4J2-323) ThreadLocal-leak on tomcat shutdown when using async logging

     [ https://issues.apache.org/jira/browse/LOG4J2-323?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Remko Popma updated LOG4J2-323:
-------------------------------
    Comment: was deleted

(was: Changes to support configuration file with AsyncRoot/AsyncLogger when web container classpath contains the jar files.
{code}
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 @@
             final boolean includeLocation) {
         super(name, appenders, filter, level, additive, properties, config,
                 includeLocation);
+        helper = config.getAsyncLoggerConfigDelegate();
     }
 
     /**
@@ -109,20 +110,12 @@
     @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();
     }
 
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 @@
  * 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}.
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..c91c9b7 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 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 @@
  * 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 @@
     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;
 
-    private static volatile int count = 0;
-    private static ThreadLocal<Boolean> isAppenderThread = new ThreadLocal<>();
+        /**
+         * 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;
+            }
+        }
+    }
 
     /**
      * Factory used to populate the RingBuffer with events. These event objects are then re-used during the life of the
@@ -82,7 +125,7 @@
     /**
      * 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 @@
         }
     };
 
+    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,84 +226,16 @@
     }
 
     /**
-     * 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 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()
@@ -279,8 +266,9 @@
     /**
      * 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 
      */
-    private static void initThreadLocalForExecutorThread() {
+    private static void initThreadLocalForExecutorThread(final ExecutorService executor) {
         executor.submit(new Runnable() {
             @Override
             public void run() {
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerContext.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerContext.java
index 70c3568..7892f56 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerContext.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerContext.java
@@ -73,7 +73,7 @@
      */
     @Override
     public void start() {
-    	maybeStartHelper(getConfiguration());
+    	helper.start();
         super.start();
     }
 
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..3c7e959 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
@@ -36,7 +36,6 @@
 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 +43,13 @@
 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 +59,10 @@
 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 +123,7 @@
     private ScriptManager scriptManager;
     private ScheduledExecutorService executorService;
     private final WatchManager watchManager = new WatchManager();
+    private final AsyncLoggerConfigHelper asyncLoggerConfigHelper = new AsyncLoggerConfigHelper();
 
     /**
      * Constructor.
@@ -166,6 +164,11 @@
     public ScheduledExecutorService getExecutorService() {
         return executorService;
     }
+
+	@Override
+	public AsyncLoggerConfigDelegate getAsyncLoggerConfigDelegate() {
+		return asyncLoggerConfigHelper;
+	}
 
     /**
      * Initialize the configuration.
@@ -215,6 +218,9 @@
         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 +236,16 @@
         LOGGER.debug("Started configuration {} OK.", this);
     }
 
-    /**
+    private boolean hasAsyncLoggers() {
+        for (final LoggerConfig logger : loggers.values()) {
+            if (logger instanceof AsyncLoggerConfig) {
+            	return true;
+            }
+        }
+		return false;
+	}
+
+	/**
      * Tear down the configuration.
      */
     @Override
@@ -238,47 +253,37 @@
         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);
+
+        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 +298,12 @@
         }
         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 +311,18 @@
                 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 +332,6 @@
             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);
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,18 +16,19 @@
  */
 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 @@
 
     ScriptManager getScriptManager();
 
+    /**
+     * Returns the {@code AsyncLoggerConfigDelegate} shared by all
+     * {@code AsyncLoggerConfig} instances defined in this Configuration.
+     * 
+     * @return the {@code AsyncLoggerConfigDelegate}
+     */
+	AsyncLoggerConfigDelegate getAsyncLoggerConfigDelegate();
+
 }
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 @@
 
     @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 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();
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 @@
         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();
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 @@
         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();
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 @@
         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();
{code})

> ThreadLocal-leak on tomcat shutdown when using async logging
> ------------------------------------------------------------
>
>                 Key: LOG4J2-323
>                 URL: https://issues.apache.org/jira/browse/LOG4J2-323
>             Project: Log4j 2
>          Issue Type: Bug
>    Affects Versions: 2.0-beta9
>         Environment: Mac OS X 10.8.4, Tomcat 7.0.42, java version 1.6.0_51
>            Reporter: Michael Kloster
>            Assignee: Remko Popma
>              Labels: Async
>             Fix For: 2.3, 2.5
>
>
> When shutting down Tomcat 7.0.42, catalina.out displays the following warning indicating a memory leak:
> {code}
> Jul 28, 2013 9:55:59 AM org.apache.coyote.AbstractProtocol start
> INFO: Starting ProtocolHandler ["http-bio-8080"]
> Jul 28, 2013 9:55:59 AM org.apache.coyote.AbstractProtocol start
> INFO: Starting ProtocolHandler ["ajp-bio-8009"]
> Jul 28, 2013 9:55:59 AM org.apache.catalina.startup.Catalina start
> INFO: Server startup in 841 ms
> Jul 28, 2013 9:56:09 AM org.apache.catalina.core.StandardServer await
> INFO: A valid shutdown command was received via the shutdown port. Stopping the Server instance.
> Jul 28, 2013 9:56:09 AM org.apache.coyote.AbstractProtocol pause
> INFO: Pausing ProtocolHandler ["http-bio-8080"]
> Jul 28, 2013 9:56:09 AM org.apache.coyote.AbstractProtocol pause
> INFO: Pausing ProtocolHandler ["ajp-bio-8009"]
> Jul 28, 2013 9:56:09 AM org.apache.catalina.core.StandardService stopInternal
> INFO: Stopping service Catalina
> Jul 28, 2013 9:56:09 AM org.apache.catalina.loader.WebappClassLoader checkThreadLocalMapForLeaks
> SEVERE: The web application [/asynclog] created a ThreadLocal with key of type [java.lang.ThreadLocal] (value [java.lang.ThreadLocal@648bfdea]) and a value of type [org.apache.logging.log4j.core.async.AsyncLogger.Info] (value [org.apache.logging.log4j.core.async.AsyncLogger$Info@4e26d560]) but failed to remove it when the web application was stopped. Threads are going to be renewed over time to try and avoid a probable memory leak.
> Jul 28, 2013 9:56:09 AM org.apache.coyote.AbstractProtocol stop
> INFO: Stopping ProtocolHandler ["http-bio-8080"]
> Jul 28, 2013 9:56:09 AM org.apache.coyote.AbstractProtocol stop
> INFO: Stopping ProtocolHandler ["ajp-bio-8009"]
> Jul 28, 2013 9:56:09 AM org.apache.coyote.AbstractProtocol destroy
> INFO: Destroying ProtocolHandler ["http-bio-8080"]
> Jul 28, 2013 9:56:09 AM org.apache.coyote.AbstractProtocol destroy
> INFO: Destroying ProtocolHandler ["ajp-bio-8009"]
> {code}
> {code:xml|title=log4j2.xml|borderStyle=solid}
> <?xml version="1.0" encoding="UTF-8"?>
> <configuration status="WARN">
>   <appenders>
>     <FastRollingFile name="MyFileLog" filename="logs/my.log"
>       filePattern="logs/my-%d{COMPACT}.log">
>         <PatternLayout pattern="%d %p %c{1.} [%t] %m%n" />
> 	<Policies>
> 	  <SizeBasedTriggeringPolicy size="5MB" />
> 	</Policies>
>     </FastRollingFile>
>     <Console name="Console" target="SYSTEM_OUT">
>       <PatternLayout pattern="%d %p %c{1.} [%t] %m%n" />
>     </Console>
>   </appenders>
>   <loggers>
>     <logger name="mylogger" level="INFO" additivity="false">
>       <appender-ref ref="MyFileLog" />
>     </logger>
>     <root level="TRACE">
>       <appender-ref ref="Console" />
>     </root>
>   </loggers>
> </configuration>
> {code}
> {code:title=log4j2.xml|borderStyle=solid}
> #!/bin/bash
> CATALINA_OPTS=-DLog4jContextSelector=org.apache.logging.log4j.core.async.AsyncLoggerContextSelector
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: log4j-dev-unsubscribe@logging.apache.org
For additional commands, e-mail: log4j-dev-help@logging.apache.org