You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@logging.apache.org by ck...@apache.org on 2018/05/01 02:02:59 UTC

[3/3] logging-log4j2 git commit: [LOG4J2-2301] Mixed async loggers no longer forget parameter values

[LOG4J2-2301] Mixed async loggers no longer forget parameter values

Previously each AsyncLoggerConfig would individually enqueue an
event on the async delegate disruptor. In practice this caused
us to trade reusable message parameters away at the first
AsyncLoggerConfig in our path, causing the rest to get an array
of nulls.

Now we begin by traversing the configuration and logging to all
synchronous loggers first, then enqueue the event to the highest
level asynchronous logger allowing the asynchronous loggers to be
traversed on the background thread.


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

Branch: refs/heads/release-2.x
Commit: b34d8cc585368db51cc84e916a257d55de44e8d0
Parents: 2446dcf
Author: Carter Kozak <ck...@apache.org>
Authored: Wed Apr 11 15:53:01 2018 -0700
Committer: Carter Kozak <ck...@apache.org>
Committed: Mon Apr 30 21:45:13 2018 -0400

----------------------------------------------------------------------
 .../log4j/core/async/AsyncLoggerConfig.java     | 59 ++++++++++++++------
 .../core/async/AsyncLoggerConfigDisruptor.java  |  2 +-
 .../logging/log4j/core/async/EventRoute.java    |  4 +-
 .../logging/log4j/core/config/LoggerConfig.java | 50 ++++++++++++++---
 .../core/async/AsyncLoggerConfigTest4.java      |  9 ++-
 .../test/resources/AsyncLoggerConfigTest4.xml   |  8 ++-
 src/changes/changes.xml                         |  3 +
 7 files changed, 106 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/b34d8cc5/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 8a68ed0..f32f0a7 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
@@ -72,6 +72,7 @@ import org.apache.logging.log4j.util.Strings;
 @Plugin(name = "asyncLogger", category = Node.CATEGORY, printObject = true)
 public class AsyncLoggerConfig extends LoggerConfig {
 
+    private static final ThreadLocal<Boolean> ASYNC_LOGGER_ENTERED = new ThreadLocal<>();
     private final AsyncLoggerConfigDelegate delegate;
 
     protected AsyncLoggerConfig(final String name,
@@ -85,16 +86,41 @@ public class AsyncLoggerConfig extends LoggerConfig {
         delegate.setLogEventFactory(getLogEventFactory());
     }
 
-    /**
-     * Passes on the event to a separate thread that will call
-     * {@link #asyncCallAppenders(LogEvent)}.
-     */
+    protected void log(final LogEvent event, final LoggerConfigPredicate predicate) {
+        if (predicate == LoggerConfigPredicate.ALL && ASYNC_LOGGER_ENTERED.get() == null) { // See LOG4J2-2301
+            // This is the first AsnycLoggerConfig encountered by this LogEvent
+            ASYNC_LOGGER_ENTERED.set(Boolean.TRUE);
+            try {
+                // Detect the first time we encounter an AsyncLoggerConfig. We must log
+                // to all non-async loggers first.
+                super.log(event, LoggerConfigPredicate.SYNCHRONOUS_ONLY);
+                // Then pass the event to the background thread where
+                // all async logging is executed. It is important this
+                // happens at most once and after all synchronous loggers
+                // have been invoked, because we lose parameter references
+                // from reusable messages.
+                logToAsyncDelegate(event);
+            } finally {
+                ASYNC_LOGGER_ENTERED.remove();
+            }
+        } else {
+            super.log(event, predicate);
+        }
+    }
+
     @Override
     protected void callAppenders(final LogEvent event) {
-        populateLazilyInitializedFields(event);
+        super.callAppenders(event);
+    }
 
-        if (!delegate.tryEnqueue(event, this)) {
-            handleQueueFull(event);
+    private void logToAsyncDelegate(LogEvent event) {
+        if (!isFiltered(event)) {
+            // Passes on the event to a separate thread that will call
+            // asyncCallAppenders(LogEvent).
+            populateLazilyInitializedFields(event);
+            if (!delegate.tryEnqueue(event, this)) {
+                handleQueueFull(event);
+            }
         }
     }
 
@@ -102,7 +128,7 @@ public class AsyncLoggerConfig extends LoggerConfig {
         if (AbstractLogger.getRecursionDepth() > 1) { // LOG4J2-1518, LOG4J2-2031
             // If queue is full AND we are in a recursive call, call appender directly to prevent deadlock
             AsyncQueueFullMessageUtil.logWarningToStatusLogger();
-            callAppendersInCurrentThread(event);
+            logToAsyncLoggerConfigsOnCurrentThread(event);
         } else {
             // otherwise, we leave it to the user preference
             final EventRoute eventRoute = delegate.getEventRoute(event.getLevel());
@@ -115,17 +141,18 @@ public class AsyncLoggerConfig extends LoggerConfig {
         event.getThreadName();
     }
 
-    void callAppendersInCurrentThread(final LogEvent event) {
-        super.callAppenders(event);
-    }
-
-    void callAppendersInBackgroundThread(final LogEvent event) {
+    void logInBackgroundThread(final LogEvent event) {
         delegate.enqueueEvent(event, this);
     }
 
-    /** Called by AsyncLoggerConfigHelper.RingBufferLog4jEventHandler. */
-    void asyncCallAppenders(final LogEvent event) {
-        super.callAppenders(event);
+    /**
+     * Called by AsyncLoggerConfigHelper.RingBufferLog4jEventHandler.
+     *
+     * This method will log the provided event to only configs of type {@link AsyncLoggerConfig} (not
+     * default {@link LoggerConfig} definitions), which will be invoked on the <b>calling thread</b>.
+     */
+    void logToAsyncLoggerConfigsOnCurrentThread(final LogEvent event) {
+        log(event, LoggerConfigPredicate.ASYNCHRONOUS_ONLY);
     }
 
     private String displayName() {

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/b34d8cc5/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDisruptor.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDisruptor.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDisruptor.java
index cc3e77b..3f6a91e 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDisruptor.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigDisruptor.java
@@ -108,7 +108,7 @@ public class AsyncLoggerConfigDisruptor extends AbstractLifeCycle implements Asy
         public void onEvent(final Log4jEventWrapper event, final long sequence, final boolean endOfBatch)
                 throws Exception {
             event.event.setEndOfBatch(endOfBatch);
-            event.loggerConfig.asyncCallAppenders(event.event);
+            event.loggerConfig.logToAsyncLoggerConfigsOnCurrentThread(event.event);
             event.clear();
 
             notifyIntermediateProgress(sequence);

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/b34d8cc5/log4j-core/src/main/java/org/apache/logging/log4j/core/async/EventRoute.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/EventRoute.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/EventRoute.java
index a1b9bb6..cdb5451 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/EventRoute.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/EventRoute.java
@@ -43,7 +43,7 @@ public enum EventRoute {
 
         @Override
         public void logMessage(final AsyncLoggerConfig asyncLoggerConfig, final LogEvent event) {
-            asyncLoggerConfig.callAppendersInBackgroundThread(event);
+            asyncLoggerConfig.logInBackgroundThread(event);
         }
 
         @Override
@@ -62,7 +62,7 @@ public enum EventRoute {
 
         @Override
         public void logMessage(final AsyncLoggerConfig asyncLoggerConfig, final LogEvent event) {
-            asyncLoggerConfig.callAppendersInCurrentThread(event);
+            asyncLoggerConfig.logToAsyncLoggerConfigsOnCurrentThread(event);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/b34d8cc5/log4j-core/src/main/java/org/apache/logging/log4j/core/config/LoggerConfig.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/config/LoggerConfig.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/config/LoggerConfig.java
index b0c12b1..d3bb03a 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/config/LoggerConfig.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/config/LoggerConfig.java
@@ -31,6 +31,7 @@ import org.apache.logging.log4j.core.Core;
 import org.apache.logging.log4j.core.Filter;
 import org.apache.logging.log4j.core.LogEvent;
 import org.apache.logging.log4j.core.LoggerContext;
+import org.apache.logging.log4j.core.async.AsyncLoggerConfig;
 import org.apache.logging.log4j.core.async.AsyncLoggerContext;
 import org.apache.logging.log4j.core.async.AsyncLoggerContextSelector;
 import org.apache.logging.log4j.core.config.plugins.Plugin;
@@ -402,7 +403,7 @@ public class LoggerConfig extends AbstractFilterable {
         }
         final LogEvent logEvent = logEventFactory.createEvent(loggerName, marker, fqcn, level, data, props, t);
         try {
-            log(logEvent);
+            log(logEvent, LoggerConfigPredicate.ALL);
         } finally {
             // LOG4J2-1583 prevent scrambled logs when logging calls are nested (logging in toString())
             ReusableLogEventFactory.release(logEvent);
@@ -415,8 +416,19 @@ public class LoggerConfig extends AbstractFilterable {
      * @param event The log event.
      */
     public void log(final LogEvent event) {
+        log(event, LoggerConfigPredicate.ALL);
+    }
+
+    /**
+     * Logs an event.
+     *
+     * @param event The log event.
+     * @param predicate predicate for which LoggerConfig instances to append to.
+     *                  A null value is equivalent to a true predicate.
+     */
+    protected void log(final LogEvent event, final LoggerConfigPredicate predicate) {
         if (!isFiltered(event)) {
-            processLogEvent(event);
+            processLogEvent(event, predicate);
         }
     }
 
@@ -430,15 +442,17 @@ public class LoggerConfig extends AbstractFilterable {
         return reliabilityStrategy;
     }
 
-    private void processLogEvent(final LogEvent event) {
+    private void processLogEvent(final LogEvent event, LoggerConfigPredicate predicate) {
         event.setIncludeLocation(isIncludeLocation());
-        callAppenders(event);
-        logParent(event);
+        if (predicate.allow(this)) {
+            callAppenders(event);
+        }
+        logParent(event, predicate);
     }
 
-    private void logParent(final LogEvent event) {
+    private void logParent(final LogEvent event, final LoggerConfigPredicate predicate) {
         if (additive && parent != null) {
-            parent.log(event);
+            parent.log(event, predicate);
         }
     }
 
@@ -577,4 +591,26 @@ public class LoggerConfig extends AbstractFilterable {
         }
     }
 
+    protected enum LoggerConfigPredicate {
+        ALL() {
+            @Override
+            boolean allow(LoggerConfig config) {
+                return true;
+            }
+        },
+        ASYNCHRONOUS_ONLY() {
+            @Override
+            boolean allow(LoggerConfig config) {
+                return config instanceof AsyncLoggerConfig;
+            }
+        },
+        SYNCHRONOUS_ONLY() {
+            @Override
+            boolean allow(LoggerConfig config) {
+                return !ASYNCHRONOUS_ONLY.allow(config);
+            }
+        };
+
+        abstract boolean allow(LoggerConfig config);
+    }
 }

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/b34d8cc5/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigTest4.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigTest4.java b/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigTest4.java
index 2b0df0b..f4f6764 100644
--- a/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigTest4.java
+++ b/log4j-core/src/test/java/org/apache/logging/log4j/core/async/AsyncLoggerConfigTest4.java
@@ -23,7 +23,6 @@ import org.apache.logging.log4j.core.CoreLoggerContexts;
 import org.apache.logging.log4j.core.config.ConfigurationFactory;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
@@ -32,6 +31,7 @@ import java.io.File;
 import java.io.FileReader;
 
 import static org.hamcrest.Matchers.containsString;
+import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
@@ -50,7 +50,6 @@ public class AsyncLoggerConfigTest4 {
     }
 
     @Test
-    @Ignore("Ignored until LOG4J2-2301 is resolved")
     public void testParameters() throws Exception {
         final File file = new File("target", "AsyncLoggerConfigTest4.log");
         assertTrue("Deleted old file before test", !file.exists() || file.delete());
@@ -62,10 +61,16 @@ public class AsyncLoggerConfigTest4 {
         final BufferedReader reader = new BufferedReader(new FileReader(file));
         final String line1 = reader.readLine();
         final String line2 = reader.readLine();
+        final String line3 = reader.readLine();
+        final String line4 = reader.readLine();
+        final String line5 = reader.readLine();
         reader.close();
         file.delete();
 
         assertThat(line1, containsString("Additive logging: {} for the price of {}! [2,1] Additive logging: 2 for the price of 1!"));
         assertThat(line2, containsString("Additive logging: {} for the price of {}! [2,1] Additive logging: 2 for the price of 1!"));
+        assertThat(line3, containsString("Additive logging: {} for the price of {}! [2,1] Additive logging: 2 for the price of 1!"));
+        assertThat(line4, containsString("Additive logging: {} for the price of {}! [2,1] Additive logging: 2 for the price of 1!"));
+        assertNull("Expected only two lines to be logged", line5);
     }
 }

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/b34d8cc5/log4j-core/src/test/resources/AsyncLoggerConfigTest4.xml
----------------------------------------------------------------------
diff --git a/log4j-core/src/test/resources/AsyncLoggerConfigTest4.xml b/log4j-core/src/test/resources/AsyncLoggerConfigTest4.xml
index 22bfd43..0af1a3f 100644
--- a/log4j-core/src/test/resources/AsyncLoggerConfigTest4.xml
+++ b/log4j-core/src/test/resources/AsyncLoggerConfigTest4.xml
@@ -12,9 +12,15 @@
     </RandomAccessFile>
   </Appenders>
   <Loggers>
-    <AsyncLogger name="com.foo.Bar" level="trace">
+    <Logger name="com.foo.Bar" level="trace">
+      <AppenderRef ref="File"/>
+    </Logger>
+    <AsyncLogger name="com.foo" level="trace">
       <AppenderRef ref="File"/>
     </AsyncLogger>
+    <Logger name="com" level="trace">
+      <AppenderRef ref="File"/>
+    </Logger>
     <AsyncRoot level="info">
       <AppenderRef ref="File"/>
     </AsyncRoot>

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/b34d8cc5/src/changes/changes.xml
----------------------------------------------------------------------
diff --git a/src/changes/changes.xml b/src/changes/changes.xml
index 3baaa5b..d2cad4c 100644
--- a/src/changes/changes.xml
+++ b/src/changes/changes.xml
@@ -84,6 +84,9 @@
       <action issue="LOG4J2-2328" dev="ggregory" type="update">
         Update JAnsi from 1.17 to 1.17.1.
       </action>
+      <action issue="LOG4J2-2301" dev="ckozak" type="fix">
+        Mixed async loggers no longer forget parameter values, providing some appenders with an array of nulls.
+      </action>
     </release>
     <release version="2.11.0" date="2018-03-11" description="GA Release 2.11.0">
       <action issue="LOG4J2-2104" dev="rgoers" type="fix">