You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@logging.apache.org by rg...@apache.org on 2016/03/16 01:21:09 UTC

[25/38] logging-log4j2 git commit: LOG4J2-1080 AsyncLogger fixes and optimizations

LOG4J2-1080 AsyncLogger fixes and optimizations

- AsyncLogger now tries to publish to the ring buffer and falls back to AsyncEventRouter when the queue is full
- falling back to AsyncEventRouter no longer calls EventRoute.logMessage() but internally switches on the enum to avoid unnecessarily re-populating the translator
- EventRoute.logMessage() implementations for AsyncLogger have been removed
- simplified RingBufferLogEventTranslator initialization
- RingBufferLogEventTranslator thread-related fields are not modified by default unless ThreadNameCachingStrategy is UNCACHED
- optimization for determining whether location should be calculated or not


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

Branch: refs/heads/LOG4J2-1116
Commit: a8bbdcd1f355e98bc89c96c02e057380836dd6e0
Parents: 43cefdd
Author: rpopma <rp...@apache.org>
Authored: Tue Mar 15 02:26:36 2016 +1100
Committer: rpopma <rp...@apache.org>
Committed: Tue Mar 15 02:26:36 2016 +1100

----------------------------------------------------------------------
 .../logging/log4j/core/async/AsyncLogger.java   | 165 ++++++++-----------
 .../logging/log4j/core/async/EventRoute.java    |   2 -
 2 files changed, 68 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/a8bbdcd1/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLogger.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLogger.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLogger.java
index 226ba08..8817b29 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLogger.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/async/AsyncLogger.java
@@ -74,6 +74,8 @@ public class AsyncLogger extends Logger implements EventTranslatorVararg<RingBuf
     private final ThreadLocal<RingBufferLogEventTranslator> threadLocalTranslator = new ThreadLocal<>();
     private final AsyncLoggerDisruptor loggerDisruptor;
 
+    // Reconfigurable. Volatile field nanoClock is always updated later, so no need to make includeLocation volatile.
+    private boolean includeLocation;
     private volatile NanoClock nanoClock; // reconfigurable
 
     /**
@@ -88,6 +90,7 @@ public class AsyncLogger extends Logger implements EventTranslatorVararg<RingBuf
             final AsyncLoggerDisruptor loggerDisruptor) {
         super(context, name, messageFactory);
         this.loggerDisruptor = loggerDisruptor;
+        includeLocation = privateConfig.loggerConfig.isIncludeLocation();
         nanoClock = context.getConfiguration().getNanoClock();
     }
 
@@ -99,6 +102,7 @@ public class AsyncLogger extends Logger implements EventTranslatorVararg<RingBuf
     @Override
     protected void updateConfiguration(Configuration newConfig) {
         super.updateConfiguration(newConfig);
+        includeLocation = privateConfig.loggerConfig.isIncludeLocation();
         nanoClock = newConfig.getNanoClock();
         LOGGER.trace("[{}] AsyncLogger {} uses {}.", getContext().getName(), getName(), nanoClock);
     }
@@ -121,66 +125,10 @@ public class AsyncLogger extends Logger implements EventTranslatorVararg<RingBuf
     public void logMessage(final String fqcn, final Level level, final Marker marker, final Message message,
             final Throwable thrown) {
 
-        // Implementation note: this method is tuned for performance. MODIFY WITH CARE!
-
-        final EventRoute eventRoute = loggerDisruptor.getEventRoute(level);
-        eventRoute.logMessage(this, fqcn, level, marker, message, thrown);
-    }
-
-    /**
-     * LOG4J2-471: prevent deadlock when RingBuffer is full and object being logged calls Logger.log() from its
-     * toString() method
-     *
-     * @param fqcn fully qualified caller name
-     * @param level log level
-     * @param marker optional marker
-     * @param message log message
-     * @param thrown optional exception
-     */
-    void logMessageInCurrentThread(final String fqcn, final Level level, final Marker marker,
-            final Message message, final Throwable thrown) {
-        // bypass RingBuffer and invoke Appender directly
-        final ReliabilityStrategy strategy = privateConfig.loggerConfig.getReliabilityStrategy();
-        strategy.log(this, getName(), fqcn, marker, level, message, thrown);
-    }
-
-    /**
-     * Enqueues the specified message to be logged in the background thread.
-     *
-     * @param fqcn fully qualified caller name
-     * @param level log level
-     * @param marker optional marker
-     * @param message log message
-     * @param thrown optional exception
-     */
-    void logMessageInBackgroundThread(final String fqcn, final Level level, final Marker marker,
-            final Message message, final Throwable thrown) {
-
-        // Implementation note: this method is tuned for performance. MODIFY WITH CARE!
-
         // if the Message instance is reused, there is no point in freezing its message here
-        if (!isReused(message) && !Constants.FORMAT_MESSAGES_IN_BACKGROUND) { // LOG4J2-898: user may choose
+        if (!Constants.FORMAT_MESSAGES_IN_BACKGROUND && !isReused(message)) { // LOG4J2-898: user may choose
             message.getFormattedMessage(); // LOG4J2-763: ask message to freeze parameters
         }
-        logInBackground(fqcn, level, marker, message, thrown);
-    }
-
-    private boolean isReused(final Message message) {
-        return message instanceof ReusableMessage;
-    }
-
-    /**
-     * Enqueues the specified log event data for logging in a background thread.
-     *
-     * @param fqcn fully qualified name of the caller
-     * @param level level at which the caller wants to log the message
-     * @param marker message marker
-     * @param message the log message
-     * @param thrown a {@code Throwable} or {@code null}
-     */
-    private void logInBackground(final String fqcn, final Level level, final Marker marker, final Message message,
-            final Throwable thrown) {
-        // Implementation note: this method is tuned for performance. MODIFY WITH CARE!
 
         if (loggerDisruptor.isUseThreadLocals()) {
             logWithThreadLocalTranslator(fqcn, level, marker, message, thrown);
@@ -190,6 +138,10 @@ public class AsyncLogger extends Logger implements EventTranslatorVararg<RingBuf
         }
     }
 
+    private boolean isReused(final Message message) {
+        return message instanceof ReusableMessage;
+    }
+
     /**
      * Enqueues the specified log event data for logging in a background thread.
      * <p>
@@ -208,35 +160,35 @@ public class AsyncLogger extends Logger implements EventTranslatorVararg<RingBuf
 
         final RingBufferLogEventTranslator translator = getCachedTranslator();
         initTranslator(translator, fqcn, level, marker, message, thrown);
-        loggerDisruptor.enqueueLogMessageInfo(translator);
+        if (!loggerDisruptor.tryPublish(translator)) {
+            handleRingBufferFull(translator);
+        }
     }
 
-    private void initTranslator(final RingBufferLogEventTranslator translator, final String fqcn,
-            final Level level, final Marker marker, final Message message, final Throwable thrown) {
-
-        // Implementation note: this method is tuned for performance. MODIFY WITH CARE!
-
-        initTranslatorPart1(translator, fqcn, level, marker, message, thrown);
-        initTranslatorPart2(translator, fqcn, message);
+    private void handleRingBufferFull(final RingBufferLogEventTranslator translator) {
+        final EventRoute eventRoute = loggerDisruptor.getEventRoute(translator.level);
+        switch (eventRoute) {
+            case ENQUEUE:
+                loggerDisruptor.enqueueLogMessageInfo(translator);
+                break;
+            case SYNCHRONOUS:
+                logMessageInCurrentThread(translator.fqcn, translator.level, translator.marker, translator.message,
+                        translator.thrown);
+                break;
+            case DISCARD:
+                break;
+            default:
+                throw new IllegalStateException("Unknown EventRoute " + eventRoute);
+        }
     }
 
-    private void initTranslatorPart1(final RingBufferLogEventTranslator translator, final String fqcn,
+    private void initTranslator(final RingBufferLogEventTranslator translator, final String fqcn,
             final Level level, final Marker marker, final Message message, final Throwable thrown) {
 
-        // Implementation note: this method is tuned for performance. MODIFY WITH CARE!
-
-        translator.setValuesPart1(this, getName(), marker, fqcn, level, message, //
+        translator.setBasicValues(this, name, marker, fqcn, level, message, //
                 // don't construct ThrowableProxy until required
-                thrown);
-    }
-
-    private void initTranslatorPart2(final RingBufferLogEventTranslator translator, final String fqcn,
-            final Message message) {
+                thrown,
 
-        // Implementation note: this method is tuned for performance. MODIFY WITH CARE!
-
-        final Thread currentThread = Thread.currentThread();
-        translator.setValuesPart2(
                 // config properties are taken care of in the EventHandler thread
                 // in the AsyncLogger#actualAsyncLog method
 
@@ -245,17 +197,33 @@ public class AsyncLogger extends Logger implements EventTranslatorVararg<RingBuf
 
                 // needs shallow copy to be fast (LOG4J2-154)
                 ThreadContext.getImmutableStack(), //
-
-                currentThread.getId(), //
-
-                // Thread.currentThread().getName(), //
-                THREAD_NAME_CACHING_STRATEGY.getThreadName(),
-
-                currentThread.getPriority(), //
                 // location (expensive to calculate)
-                calcLocationIfRequested(fqcn),
-                eventTimeMillis(message), nanoClock.nanoTime() //
-                );
+                calcLocationIfRequested(fqcn), //
+                eventTimeMillis(message), //
+                nanoClock.nanoTime() //
+        );
+        // constant check should be optimized out when using default (CACHED)
+        if (THREAD_NAME_CACHING_STRATEGY == ThreadNameCachingStrategy.UNCACHED) {
+            final Thread currentThread = Thread.currentThread();
+            translator.setThreadValues(
+                    currentThread.getId(), //
+                    currentThread.getName(), //
+                    currentThread.getPriority() //
+            );
+        }
+    }
+
+    /**
+     * Returns the caller location if requested, {@code null} otherwise.
+     *
+     * @param fqcn fully qualified caller name.
+     * @return the caller location if requested, {@code null} otherwise.
+     */
+    private StackTraceElement calcLocationIfRequested(String fqcn) {
+        // location: very expensive operation. LOG4J2-153:
+        // Only include if "includeLocation=true" is specified,
+        // exclude if not specified or if "false" was specified.
+        return includeLocation ? Log4jLogEvent.calcLocation(fqcn) : null;
     }
 
     private long eventTimeMillis(final Message message) {
@@ -325,17 +293,20 @@ public class AsyncLogger extends Logger implements EventTranslatorVararg<RingBuf
     }
 
     /**
-     * Returns the caller location if requested, {@code null} otherwise.
+     * LOG4J2-471: prevent deadlock when RingBuffer is full and object being logged calls Logger.log() from its
+     * toString() method
      *
-     * @param fqcn fully qualified caller name.
-     * @return the caller location if requested, {@code null} otherwise.
+     * @param fqcn fully qualified caller name
+     * @param level log level
+     * @param marker optional marker
+     * @param message log message
+     * @param thrown optional exception
      */
-    private StackTraceElement calcLocationIfRequested(String fqcn) {
-        // location: very expensive operation. LOG4J2-153:
-        // Only include if "includeLocation=true" is specified,
-        // exclude if not specified or if "false" was specified.
-        final boolean includeLocation = privateConfig.loggerConfig.isIncludeLocation();
-        return includeLocation ? Log4jLogEvent.calcLocation(fqcn) : null;
+    void logMessageInCurrentThread(final String fqcn, final Level level, final Marker marker,
+            final Message message, final Throwable thrown) {
+        // bypass RingBuffer and invoke Appender directly
+        final ReliabilityStrategy strategy = privateConfig.loggerConfig.getReliabilityStrategy();
+        strategy.log(this, getName(), fqcn, marker, level, message, thrown);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/a8bbdcd1/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 c778196..ce57008 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
@@ -36,7 +36,6 @@ public enum EventRoute {
         @Override
         public void logMessage(final AsyncLogger asyncLogger, final String fqcn, final Level level,
                 final Marker marker, final Message message, final Throwable thrown) {
-            asyncLogger.logMessageInBackgroundThread(fqcn, level, marker, message, thrown);
         }
 
         @Override
@@ -56,7 +55,6 @@ public enum EventRoute {
         @Override
         public void logMessage(final AsyncLogger asyncLogger, final String fqcn, final Level level,
                 final Marker marker, final Message message, final Throwable thrown) {
-            asyncLogger.logMessageInCurrentThread(fqcn, level, marker, message, thrown);
         }
 
         @Override