You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@logging.apache.org by rp...@apache.org on 2017/06/18 07:06:47 UTC

logging-log4j2 git commit: LOG4J2-1874 add writeBytes methods to ByteBufferDestination

Repository: logging-log4j2
Updated Branches:
  refs/heads/master 749ec08b4 -> d92cc27d3


LOG4J2-1874 add writeBytes methods to ByteBufferDestination

Added methods ::writeBytes(ByteBuffer) and ::writeBytes(byte[], int, int) to ByteBufferDestination interface and use these methods in TextEncoderHelper where possible to prepare for future enhancements to reduce lock contention.
Closes GitHub pull request #71


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

Branch: refs/heads/master
Commit: d92cc27d3cdc0a5d7e7f8c519abf592c547e015b
Parents: 749ec08
Author: rpopma <rp...@apache.org>
Authored: Sun Jun 18 16:06:37 2017 +0900
Committer: rpopma <rp...@apache.org>
Committed: Sun Jun 18 16:06:37 2017 +0900

----------------------------------------------------------------------
 .../core/appender/OutputStreamManager.java      |  24 +-
 .../log4j/core/layout/AbstractLayout.java       |  27 +-
 .../core/layout/ByteBufferDestination.java      |  23 ++
 .../layout/ByteBufferDestinationHelper.java     |  68 +++++
 .../layout/LockingStringBuilderEncoder.java     |  14 +-
 .../log4j/core/layout/StringBuilderEncoder.java |  96 +++----
 .../log4j/core/layout/TextEncoderHelper.java    | 270 +++++++++++++------
 .../appender/MemoryMappedFileManagerTest.java   |   2 +-
 .../appender/RandomAccessFileManagerTest.java   |   4 +-
 .../RollingRandomAccessFileManagerTest.java     |   4 +-
 .../log4j/core/layout/PatternLayoutTest.java    |  17 +-
 .../core/layout/SpyByteBufferDestination.java   |  10 +
 .../core/layout/StringBuilderEncoderTest.java   |  39 ++-
 .../test/appender/EncodingListAppender.java     |  10 +
 ...ractStringLayoutStringEncodingBenchmark.java |  17 +-
 .../perf/jmh/TextEncoderHelperBenchmark.java    |  13 +-
 .../logging/log4j/perf/nogc/DemoAppender.java   |  17 +-
 .../logging/log4j/perf/util/DemoAppender.java   |  11 +
 src/changes/changes.xml                         |   3 +
 19 files changed, 476 insertions(+), 193 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/main/java/org/apache/logging/log4j/core/appender/OutputStreamManager.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/appender/OutputStreamManager.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/appender/OutputStreamManager.java
index bfce7de..60d63a7 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/appender/OutputStreamManager.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/appender/OutputStreamManager.java
@@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
 import org.apache.logging.log4j.core.Layout;
 import org.apache.logging.log4j.core.LoggerContext;
 import org.apache.logging.log4j.core.layout.ByteBufferDestination;
+import org.apache.logging.log4j.core.layout.ByteBufferDestinationHelper;
 import org.apache.logging.log4j.core.util.Constants;
 
 /**
@@ -118,7 +119,7 @@ public class OutputStreamManager extends AbstractManager implements ByteBufferDe
     protected OutputStream createOutputStream() throws IOException {
         throw new IllegalStateException(getClass().getCanonicalName() + " must implement createOutputStream()");
     }
-    
+
     /**
      * Indicate whether the footer should be skipped or not.
      * @param skipFooter true if the footer should be skipped.
@@ -201,6 +202,11 @@ public class OutputStreamManager extends AbstractManager implements ByteBufferDe
         write(bytes, 0, bytes.length, immediateFlush);
     }
 
+    @Override
+    public void writeBytes(final byte[] data, final int offset, final int length) {
+        write(data, offset, length, false);
+    }
+
     /**
      * Some output streams synchronize writes while others do not. Synchronizing here insures that
      * log events won't be intertwined.
@@ -210,7 +216,7 @@ public class OutputStreamManager extends AbstractManager implements ByteBufferDe
      * @throws AppenderLoggingException if an error occurs.
      */
     protected void write(final byte[] bytes, final int offset, final int length) {
-        write(bytes, offset, length, false);
+        writeBytes(bytes, offset, length);
     }
 
     /**
@@ -284,8 +290,8 @@ public class OutputStreamManager extends AbstractManager implements ByteBufferDe
      */
     protected synchronized void flushBuffer(final ByteBuffer buf) {
         buf.flip();
-        if (buf.limit() > 0) {
-            writeToDestination(buf.array(), 0, buf.limit());
+        if (buf.remaining() > 0) {
+            writeToDestination(buf.array(), buf.arrayOffset() + buf.position(), buf.remaining());
         }
         buf.clear();
     }
@@ -345,4 +351,14 @@ public class OutputStreamManager extends AbstractManager implements ByteBufferDe
         flushBuffer(buf);
         return buf;
     }
+
+    @Override
+    public void writeBytes(ByteBuffer data) {
+        if (data.remaining() == 0) {
+          return;
+        }
+        synchronized (this) {
+          ByteBufferDestinationHelper.writeToUnsynchronized(data, this);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/AbstractLayout.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/AbstractLayout.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/AbstractLayout.java
index 393f8c1..c8bb1be 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/AbstractLayout.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/AbstractLayout.java
@@ -17,7 +17,6 @@
 package org.apache.logging.log4j.core.layout;
 
 import java.io.Serializable;
-import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -209,30 +208,6 @@ public abstract class AbstractLayout<T extends Serializable> implements Layout<T
     @Override
     public void encode(final LogEvent event, final ByteBufferDestination destination) {
         final byte[] data = toByteArray(event);
-        writeTo(data, 0, data.length, destination);
-    }
-
-    /**
-     * Writes the specified data to the specified destination.
-     *
-     * @param data the data to write
-     * @param offset where to start in the specified data array
-     * @param length the number of bytes to write
-     * @param destination the {@code ByteBufferDestination} to write to
-     */
-    public static void writeTo(final byte[] data, int offset, int length, final ByteBufferDestination destination) {
-        int chunk = 0;
-        synchronized (destination) {
-            ByteBuffer buffer = destination.getByteBuffer();
-            do {
-                if (length > buffer.remaining()) {
-                    buffer = destination.drain(buffer);
-                }
-                chunk = Math.min(length, buffer.remaining());
-                buffer.put(data, offset, chunk);
-                offset += chunk;
-                length -= chunk;
-            } while (length > 0);
-        }
+        destination.writeBytes(data, 0, data.length);
     }
 }

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/ByteBufferDestination.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/ByteBufferDestination.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/ByteBufferDestination.java
index 88b8464..5559698 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/ByteBufferDestination.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/ByteBufferDestination.java
@@ -18,6 +18,8 @@ package org.apache.logging.log4j.core.layout;
 
 import java.nio.ByteBuffer;
 
+import org.apache.logging.log4j.core.appender.OutputStreamManager;
+
 /**
  * ByteBufferDestination is the destination that {@link Encoder}s write binary data to. It encapsulates a
  * {@code ByteBuffer} and a {@code drain()} method the producer can call when the {@code ByteBuffer} is full.
@@ -44,4 +46,25 @@ public interface ByteBufferDestination {
      * @return a buffer with more available space (which may or may not be the same instance)
      */
     ByteBuffer drain(ByteBuffer buf);
+
+    /**
+     * Writes the given data to this ByteBufferDestination entirely. Call of this method should *not* be protected
+     * with synchronized on this ByteBufferDestination instance. ByteBufferDestination implementations should
+     * synchronize themselves inside this method, if needed.
+     */
+    void writeBytes(ByteBuffer data);
+
+    /**
+     * Writes the given data to this ByteBufferDestination. Call of this method should *not* be protected with
+     * synchronized on this ByteBufferDestination instance. ByteBufferDestination implementations should
+     * synchronize themselves inside this method, if needed.
+     * <p>
+     * This method should behave identically to {@code writeBytes(ByteBuffer.wrap(data, offset, length)}.
+     * It is provided to allow callers not to generate extra garbage.
+     * <p>
+     * This method is called writeBytes() to avoid clashing with {@link OutputStreamManager#write(byte[], int, int)},
+     * which might be overridden in user-defined subclasses as protected, hence adding it to interface and requiring
+     * the method to be public breaks source compatibility.
+     */
+    void writeBytes(byte[] data, int offset, int length);
 }

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/ByteBufferDestinationHelper.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/ByteBufferDestinationHelper.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/ByteBufferDestinationHelper.java
new file mode 100644
index 0000000..0102725
--- /dev/null
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/ByteBufferDestinationHelper.java
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache license, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the license for the specific language governing permissions and
+ * limitations under the license.
+ */
+package org.apache.logging.log4j.core.layout;
+
+import java.nio.ByteBuffer;
+
+public final class ByteBufferDestinationHelper {
+
+    private ByteBufferDestinationHelper() {
+    }
+
+    /**
+     * Writes the specified data to the specified destination. Doesn't synchronize on the destination object. The helper
+     * method for {@link ByteBufferDestination#writeBytes(ByteBuffer)} implementations.
+     *
+     * @param source        the data to write
+     * @param destination the {@code ByteBufferDestination} to write to
+     */
+    public static void writeToUnsynchronized(ByteBuffer source, ByteBufferDestination destination) {
+        ByteBuffer destBuff = destination.getByteBuffer();
+        while (source.remaining() > destBuff.remaining()) {
+            int originalLimit = source.limit();
+            source.limit(Math.min(source.limit(), source.position() + destBuff.remaining()));
+            destBuff.put(source);
+            source.limit(originalLimit);
+            destBuff = destination.drain(destBuff);
+        }
+        destBuff.put(source);
+        // No drain in the end.
+    }
+
+    /**
+     * Writes the specified data to the specified destination. Doesn't synchronize on the destination object. The helper
+     * method for {@link ByteBufferDestination#writeBytes(byte[], int, int)} implementations.
+     *
+     * @param data        the data to write
+     * @param offset      where to start in the specified data array
+     * @param length      the number of bytes to write
+     * @param destination the {@code ByteBufferDestination} to write to
+     */
+    public static void writeToUnsynchronized(final byte[] data, int offset, int length,
+            final ByteBufferDestination destination) {
+        ByteBuffer buffer = destination.getByteBuffer();
+        while (length > buffer.remaining()) {
+            int chunk = buffer.remaining();
+            buffer.put(data, offset, chunk);
+            offset += chunk;
+            length -= chunk;
+            buffer = destination.drain(buffer);
+        }
+        buffer.put(data, offset, length);
+        // No drain in the end.
+    }
+}

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/LockingStringBuilderEncoder.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/LockingStringBuilderEncoder.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/LockingStringBuilderEncoder.java
index 90ab6db..5bb9a0a 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/LockingStringBuilderEncoder.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/LockingStringBuilderEncoder.java
@@ -51,15 +51,17 @@ public class LockingStringBuilderEncoder implements Encoder<StringBuilder> {
 
     @Override
     public void encode(final StringBuilder source, final ByteBufferDestination destination) {
-        synchronized (destination) {
-            try {
+        try {
+            // This synchronized is needed to be able to call destination.getByteBuffer()
+            synchronized (destination) {
                 TextEncoderHelper.encodeText(charsetEncoder, cachedCharBuffer, destination.getByteBuffer(), source,
-                        destination);
-            } catch (final Exception ex) {
-                logEncodeTextException(ex, source, destination);
-                TextEncoderHelper.encodeTextFallBack(charset, source, destination);
+                    destination);
             }
+        } catch (final Exception ex) {
+            logEncodeTextException(ex, source, destination);
+            TextEncoderHelper.encodeTextFallBack(charset, source, destination);
         }
+
     }
 
     private void logEncodeTextException(final Exception ex, final StringBuilder text,

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/StringBuilderEncoder.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/StringBuilderEncoder.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/StringBuilderEncoder.java
index 1aefc17..76aa016 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/StringBuilderEncoder.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/StringBuilderEncoder.java
@@ -32,9 +32,18 @@ import org.apache.logging.log4j.status.StatusLogger;
 public class StringBuilderEncoder implements Encoder<StringBuilder> {
 
     private static final int DEFAULT_BYTE_BUFFER_SIZE = 8 * 1024;
-    private final ThreadLocal<CharBuffer> charBufferThreadLocal = new ThreadLocal<>();
-    private final ThreadLocal<ByteBuffer> byteBufferThreadLocal = new ThreadLocal<>();
-    private final ThreadLocal<CharsetEncoder> charsetEncoderThreadLocal = new ThreadLocal<>();
+    /**
+     * This ThreadLocal uses raw and inconvenient Object[] to store three heterogeneous objects (CharEncoder, CharBuffer
+     * and ByteBuffer) instead of a custom class, because it needs to contain JDK classes, no custom (Log4j) classes.
+     * Where possible putting only JDK classes in ThreadLocals is preferable to avoid memory leaks in web containers:
+     * the Log4j classes may be loaded by a separate class loader which cannot be garbage collected if a thread pool
+     * threadlocal still has a reference to it.
+     *
+     * Using just one ThreadLocal instead of three separate ones is an optimization: {@link ThreadLocal.ThreadLocalMap}
+     * is polluted less, {@link ThreadLocal.ThreadLocalMap#get()} is called only once on each call to {@link #encode}
+     * instead of three times.
+     */
+    private final ThreadLocal<Object[]> threadLocal = new ThreadLocal<>();
     private final Charset charset;
     private final int charBufferSize;
     private final int byteBufferSize;
@@ -51,77 +60,38 @@ public class StringBuilderEncoder implements Encoder<StringBuilder> {
 
     @Override
     public void encode(final StringBuilder source, final ByteBufferDestination destination) {
-        final ByteBuffer temp = getByteBuffer();
-        temp.clear();
-        temp.limit(Math.min(temp.capacity(), destination.getByteBuffer().capacity()));
-        final CharsetEncoder charsetEncoder = getCharsetEncoder();
-
-        final int estimatedBytes = estimateBytes(source.length(), charsetEncoder.maxBytesPerChar());
-        if (temp.remaining() < estimatedBytes) {
-            encodeSynchronized(getCharsetEncoder(), getCharBuffer(), source, destination);
-        } else {
-            encodeWithThreadLocals(charsetEncoder, getCharBuffer(), temp, source, destination);
-        }
-    }
-
-    private void encodeWithThreadLocals(final CharsetEncoder charsetEncoder, final CharBuffer charBuffer,
-            final ByteBuffer temp, final StringBuilder source, final ByteBufferDestination destination) {
         try {
-            TextEncoderHelper.encodeTextWithCopy(charsetEncoder, charBuffer, temp, source, destination);
+            Object[] threadLocalState = getThreadLocalState();
+            CharsetEncoder charsetEncoder = (CharsetEncoder) threadLocalState[0];
+            CharBuffer charBuffer = (CharBuffer) threadLocalState[1];
+            ByteBuffer byteBuffer = (ByteBuffer) threadLocalState[2];
+            TextEncoderHelper.encodeText(charsetEncoder, charBuffer, byteBuffer, source, destination);
         } catch (final Exception ex) {
             logEncodeTextException(ex, source, destination);
             TextEncoderHelper.encodeTextFallBack(charset, source, destination);
         }
     }
 
-    private static int estimateBytes(final int charCount, final float maxBytesPerChar) {
-        return (int) (charCount * (double) maxBytesPerChar);
-    }
-
-    private void encodeSynchronized(final CharsetEncoder charsetEncoder, final CharBuffer charBuffer,
-            final StringBuilder source, final ByteBufferDestination destination) {
-        synchronized (destination) {
-            try {
-                TextEncoderHelper.encodeText(charsetEncoder, charBuffer, destination.getByteBuffer(), source,
-                        destination);
-            } catch (final Exception ex) {
-                logEncodeTextException(ex, source, destination);
-                TextEncoderHelper.encodeTextFallBack(charset, source, destination);
-            }
-        }
-    }
-
-    private CharsetEncoder getCharsetEncoder() {
-        CharsetEncoder result = charsetEncoderThreadLocal.get();
-        if (result == null) {
-            result = charset.newEncoder().onMalformedInput(CodingErrorAction.REPLACE)
-                    .onUnmappableCharacter(CodingErrorAction.REPLACE);
-            charsetEncoderThreadLocal.set(result);
-        }
-        return result;
-    }
-
-
-    private CharBuffer getCharBuffer() {
-        CharBuffer result = charBufferThreadLocal.get();
-        if (result == null) {
-            result = CharBuffer.wrap(new char[charBufferSize]);
-            charBufferThreadLocal.set(result);
-        }
-        return result;
-    }
-
-    private ByteBuffer getByteBuffer() {
-        ByteBuffer result = byteBufferThreadLocal.get();
-        if (result == null) {
-            result = ByteBuffer.wrap(new byte[byteBufferSize]);
-            byteBufferThreadLocal.set(result);
+    private Object[] getThreadLocalState() {
+        Object[] threadLocalState = threadLocal.get();
+        if (threadLocalState == null) {
+            threadLocalState = new Object[] {
+                    charset.newEncoder().onMalformedInput(CodingErrorAction.REPLACE)
+                            .onUnmappableCharacter(CodingErrorAction.REPLACE),
+                    CharBuffer.allocate(charBufferSize),
+                    ByteBuffer.allocate(byteBufferSize)
+            };
+            threadLocal.set(threadLocalState);
+        } else {
+            ((CharsetEncoder) threadLocalState[0]).reset();
+            ((CharBuffer) threadLocalState[1]).clear();
+            ((ByteBuffer) threadLocalState[2]).clear();
         }
-        return result;
+        return threadLocalState;
     }
 
     private void logEncodeTextException(final Exception ex, final StringBuilder text,
-                                        final ByteBufferDestination destination) {
+            final ByteBufferDestination destination) {
         StatusLogger.getLogger().error("Recovering from StringBuilderEncoder.encode('{}') error: {}", text, ex, ex);
     }
 }

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/TextEncoderHelper.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/TextEncoderHelper.java b/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/TextEncoderHelper.java
index 0b9bf92..4c6763c 100644
--- a/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/TextEncoderHelper.java
+++ b/log4j-core/src/main/java/org/apache/logging/log4j/core/layout/TextEncoderHelper.java
@@ -36,58 +36,131 @@ public class TextEncoderHelper {
     static void encodeTextFallBack(final Charset charset, final StringBuilder text,
             final ByteBufferDestination destination) {
         final byte[] bytes = text.toString().getBytes(charset);
-        synchronized (destination) {
-            ByteBuffer buffer = destination.getByteBuffer();
-            int offset = 0;
-            do {
-                final int length = Math.min(bytes.length - offset, buffer.remaining());
-                buffer.put(bytes, offset, length);
-                offset += length;
-                if (offset < bytes.length) {
-                    buffer = destination.drain(buffer);
-                }
-            } while (offset < bytes.length);
-        }
+        destination.writeBytes(bytes, 0, bytes.length);
     }
 
-    static void encodeTextWithCopy(final CharsetEncoder charsetEncoder, final CharBuffer charBuf, final ByteBuffer temp,
-            final StringBuilder text, final ByteBufferDestination destination) {
+    /**
+     * Converts the specified text to bytes and writes the resulting bytes to the specified destination.
+     * Attempts to postpone synchronizing on the destination as long as possible to minimize lock contention.
+     *
+     * @param charsetEncoder thread-local encoder instance for converting chars to bytes
+     * @param charBuf thread-local text buffer for converting text to bytes
+     * @param byteBuf thread-local buffer to temporarily hold converted bytes before copying them to the destination
+     * @param text the text to convert and write to the destination
+     * @param destination the destination to write the bytes to
+     * @throws CharacterCodingException if conversion failed
+     */
+    static void encodeText(final CharsetEncoder charsetEncoder, final CharBuffer charBuf, final ByteBuffer byteBuf,
+            final StringBuilder text, final ByteBufferDestination destination)
+            throws CharacterCodingException {
+        charsetEncoder.reset();
+        if (text.length() > charBuf.capacity()) {
+            encodeChunkedText(charsetEncoder, charBuf, byteBuf, text, destination);
+            return;
+        }
+        charBuf.clear();
+        text.getChars(0, text.length(), charBuf.array(), charBuf.arrayOffset());
+        charBuf.limit(text.length());
+        CoderResult result = charsetEncoder.encode(charBuf, byteBuf, true);
+        writeEncodedText(charsetEncoder, charBuf, byteBuf, destination, result);
+    }
 
-        encodeText(charsetEncoder, charBuf, temp, text, destination);
-        copyDataToDestination(temp, destination);
+    /**
+     * This method is called when the CharEncoder has encoded (but not yet flushed) content from the CharBuffer
+     * into the ByteBuffer. A CoderResult of UNDERFLOW means that the contents fit into the ByteBuffer and we can move
+     * on to the next step, flushing. Otherwise, we need to synchronize on the destination, copy the ByteBuffer to the
+     * destination and encode the remainder of the CharBuffer while holding the lock on the destination.
+     *
+     * @since 2.9
+     */
+    private static void writeEncodedText(final CharsetEncoder charsetEncoder, final CharBuffer charBuf,
+            final ByteBuffer byteBuf, final ByteBufferDestination destination, CoderResult result) {
+        if (!result.isUnderflow()) {
+            writeChunkedEncodedText(charsetEncoder, charBuf, destination, byteBuf, result);
+            return;
+        }
+        result = charsetEncoder.flush(byteBuf);
+        if (!result.isUnderflow()) {
+            synchronized (destination) {
+                flushRemainingBytes(charsetEncoder, destination, byteBuf);
+            }
+            return;
+        }
+        // Thread-safety note: no explicit synchronization on ByteBufferDestination below. This is safe, because
+        // if the byteBuf is actually the destination's buffer, this method call should be protected with
+        // synchronization on the destination object at some level, so the call to destination.getByteBuffer() should
+        // be safe. If the byteBuf is an unrelated buffer, the comparison between the buffers should fail despite
+        // destination.getByteBuffer() is not protected with the synchronization on the destination object.
+        if (byteBuf != destination.getByteBuffer()) {
+            byteBuf.flip();
+            destination.writeBytes(byteBuf);
+            byteBuf.clear();
+        }
     }
 
-    private static void copyDataToDestination(final ByteBuffer temp, final ByteBufferDestination destination) {
+    /**
+     * This method is called when the CharEncoder has encoded (but not yet flushed) content from the CharBuffer
+     * into the ByteBuffer and we found that the ByteBuffer is too small to hold all the content.
+     * Therefore, we need to synchronize on the destination, copy the ByteBuffer to the
+     * destination and encode the remainder of the CharBuffer while holding the lock on the destination.
+     *
+     * @since 2.9
+     */
+    private static void writeChunkedEncodedText(final CharsetEncoder charsetEncoder, final CharBuffer charBuf,
+            final ByteBufferDestination destination, ByteBuffer byteBuf, final CoderResult result) {
         synchronized (destination) {
-            ByteBuffer destinationBuffer = destination.getByteBuffer();
-            if (destinationBuffer != temp) { // still need to write to the destination
-                temp.flip();
-                if (temp.remaining() > destinationBuffer.remaining()) {
-                    destinationBuffer = destination.drain(destinationBuffer);
-                }
-                destinationBuffer.put(temp);
-                temp.clear();
-            }
+            byteBuf = writeAndEncodeAsMuchAsPossible(charsetEncoder, charBuf, true, destination, byteBuf,
+                    result);
+            flushRemainingBytes(charsetEncoder, destination, byteBuf);
         }
     }
 
-    static void encodeText(final CharsetEncoder charsetEncoder, final CharBuffer charBuf, final ByteBuffer byteBuf,
-            final StringBuilder text, final ByteBufferDestination destination) {
-        charsetEncoder.reset();
-        ByteBuffer temp = byteBuf; // may be the destination's buffer or a temporary buffer
+    /**
+     * This method is called <em>before</em> the CharEncoder has encoded any content from the CharBuffer
+     * into the ByteBuffer, but we have already detected that the CharBuffer contents is too large to fit into the
+     * ByteBuffer. Therefore, at some point we need to synchronize on the destination, copy the ByteBuffer to the
+     * destination and encode the remainder of the CharBuffer while holding the lock on the destination.
+     *
+     * @since 2.9
+     */
+    private static void encodeChunkedText(final CharsetEncoder charsetEncoder, final CharBuffer charBuf,
+            ByteBuffer byteBuf, final StringBuilder text, final ByteBufferDestination destination) {
+
+        // LOG4J2-1874 ByteBuffer, CharBuffer and CharsetEncoder are thread-local, so no need to synchronize while
+        // modifying these objects. Postpone synchronization until accessing the ByteBufferDestination.
         int start = 0;
-        int todoChars = text.length();
-        boolean endOfInput = true;
-        do {
-            charBuf.clear(); // reset character buffer position to zero, limit to capacity
+        CoderResult result = CoderResult.UNDERFLOW;
+        boolean endOfInput = false;
+        while (!endOfInput && result.isUnderflow()) {
+            charBuf.clear();
             final int copied = copy(text, start, charBuf);
             start += copied;
-            todoChars -= copied;
-            endOfInput = todoChars <= 0;
-
-            charBuf.flip(); // prepare for reading: set limit to position, position to zero
-            temp = encode(charsetEncoder, charBuf, endOfInput, destination, temp);
-        } while (!endOfInput);
+            endOfInput = start >= text.length();
+            charBuf.flip();
+            result = charsetEncoder.encode(charBuf, byteBuf, endOfInput);
+        }
+        if (endOfInput) {
+            writeEncodedText(charsetEncoder, charBuf, byteBuf, destination, result);
+            return;
+        }
+        synchronized (destination) {
+            byteBuf = writeAndEncodeAsMuchAsPossible(charsetEncoder, charBuf, endOfInput, destination, byteBuf,
+                    result);
+            while (!endOfInput) {
+                result = CoderResult.UNDERFLOW;
+                while (!endOfInput && result.isUnderflow()) {
+                    charBuf.clear();
+                    final int copied = copy(text, start, charBuf);
+                    start += copied;
+                    endOfInput = start >= text.length();
+                    charBuf.flip();
+                    result = charsetEncoder.encode(charBuf, byteBuf, endOfInput);
+                }
+                byteBuf = writeAndEncodeAsMuchAsPossible(charsetEncoder, charBuf, endOfInput, destination, byteBuf,
+                        result);
+            }
+            flushRemainingBytes(charsetEncoder, destination, byteBuf);
+        }
     }
 
     /**
@@ -96,63 +169,104 @@ public class TextEncoderHelper {
     @Deprecated
     public static void encodeText(final CharsetEncoder charsetEncoder, final CharBuffer charBuf,
             final ByteBufferDestination destination) {
+        charsetEncoder.reset();
         synchronized (destination) {
-            charsetEncoder.reset();
-            final ByteBuffer byteBuf = destination.getByteBuffer();
-            encode(charsetEncoder, charBuf, true, destination, byteBuf);
+            ByteBuffer byteBuf = destination.getByteBuffer();
+            byteBuf = encodeAsMuchAsPossible(charsetEncoder, charBuf, true, destination, byteBuf);
+            flushRemainingBytes(charsetEncoder, destination, byteBuf);
         }
     }
 
-    private static ByteBuffer encode(final CharsetEncoder charsetEncoder, final CharBuffer charBuf,
-            final boolean endOfInput, final ByteBufferDestination destination, ByteBuffer byteBuf) {
-        try {
-            byteBuf = encodeAsMuchAsPossible(charsetEncoder, charBuf, endOfInput, destination, byteBuf);
-            if (endOfInput) {
-                byteBuf = flushRemainingBytes(charsetEncoder, destination, byteBuf);
+    /**
+     * Continues to write the contents of the ByteBuffer to the destination and encode more of the CharBuffer text
+     * into the ByteBuffer until the remaining encoded text fit into the ByteBuffer, at which point the ByteBuffer
+     * is returned (without flushing the CharEncoder).
+     * <p>
+     * This method is called when the CharEncoder has encoded (but not yet flushed) content from the CharBuffer
+     * into the ByteBuffer and we found that the ByteBuffer is too small to hold all the content.
+     * </p><p>
+     * Thread-safety note: This method should be called while synchronizing on the ByteBufferDestination.
+     * </p>
+     * @return the ByteBuffer resulting from draining the temporary ByteBuffer to the destination. In the case
+     *          of a MemoryMappedFile, a remap() may have taken place and the returned ByteBuffer is now the
+     *          MappedBuffer of the newly mapped region of the memory mapped file.
+     * @since 2.9
+     */
+    private static ByteBuffer writeAndEncodeAsMuchAsPossible(final CharsetEncoder charsetEncoder,
+            final CharBuffer charBuf, final boolean endOfInput, final ByteBufferDestination destination,
+            ByteBuffer temp, CoderResult result) {
+        while (true) {
+            temp = drainIfByteBufferFull(destination, temp, result);
+            if (!result.isOverflow()) {
+                break;
             }
-        } catch (final CharacterCodingException ex) {
-            throw new IllegalStateException(ex);
+            result = charsetEncoder.encode(charBuf, temp, endOfInput);
+        }
+        if (!result.isUnderflow()) { // we should have fully read the char buffer contents
+            throwException(result);
+        }
+        return temp;
+    }
+
+    // @since 2.9
+    private static void throwException(final CoderResult result) {
+        try {
+            result.throwException();
+        } catch (CharacterCodingException e) {
+            throw new IllegalStateException(e);
         }
-        return byteBuf;
     }
 
     private static ByteBuffer encodeAsMuchAsPossible(final CharsetEncoder charsetEncoder, final CharBuffer charBuf,
-            final boolean endOfInput, final ByteBufferDestination destination, ByteBuffer temp)
-            throws CharacterCodingException {
+            final boolean endOfInput, final ByteBufferDestination destination, ByteBuffer temp) {
         CoderResult result;
         do {
             result = charsetEncoder.encode(charBuf, temp, endOfInput);
             temp = drainIfByteBufferFull(destination, temp, result);
         } while (result.isOverflow()); // byte buffer has been drained: retry
         if (!result.isUnderflow()) { // we should have fully read the char buffer contents
-            result.throwException();
+            throwException(result);
         }
         return temp;
     }
 
-    private static ByteBuffer drainIfByteBufferFull(final ByteBufferDestination destination, ByteBuffer temp, final CoderResult result) {
+    /**
+     * If the CoderResult indicates the ByteBuffer is full, synchronize on the destination and write the content
+     * of the ByteBuffer to the destination. If the specified ByteBuffer is owned by the destination, we have
+     * reached the end of a MappedBuffer and we call drain() on the destination to remap().
+     * <p>
+     * If the CoderResult indicates more can be encoded, this method does nothing and returns the temp ByteBuffer.
+     * </p>
+     *
+     * @param destination the destination to write bytes to
+     * @param temp the ByteBuffer containing the encoded bytes. May be a temporary buffer or may be the ByteBuffer of
+     *              the ByteBufferDestination
+     * @param result the CoderResult from the CharsetEncoder
+     * @return the ByteBuffer to encode into for the remainder of the text
+     */
+    private static ByteBuffer drainIfByteBufferFull(final ByteBufferDestination destination, ByteBuffer temp,
+            final CoderResult result) {
         if (result.isOverflow()) { // byte buffer full
-
-            // SHOULD NOT HAPPEN:
-            // CALLER SHOULD ONLY PASS TEMP ByteBuffer LARGE ENOUGH TO ENCODE ALL CHARACTERS,
-            // AND LOCK ON THE DESTINATION IF THIS IS NOT POSSIBLE
-            ByteBuffer destinationBuffer = destination.getByteBuffer();
-            if (destinationBuffer != temp) {
-                temp.flip();
-                destinationBuffer.put(temp);
-                temp.clear();
+            // all callers already synchronize on destination but for safety ensure we are synchronized because
+            // below calls to drain() may cause destination to swap in a new ByteBuffer object
+            synchronized (destination) {
+                ByteBuffer destinationBuffer = destination.getByteBuffer();
+                if (destinationBuffer != temp) {
+                    temp.flip();
+                    ByteBufferDestinationHelper.writeToUnsynchronized(temp, destination);
+                    temp.clear();
+                    return destination.getByteBuffer();
+                } else {
+                    return destination.drain(destinationBuffer);
+                }
             }
-            // destination consumes contents
-            // and returns byte buffer with more capacity
-            destinationBuffer = destination.drain(destinationBuffer);
-            temp = destinationBuffer;
+        } else {
+            return temp;
         }
-        return temp;
     }
 
-    private static ByteBuffer flushRemainingBytes(final CharsetEncoder charsetEncoder,
-            final ByteBufferDestination destination, ByteBuffer temp)
-            throws CharacterCodingException {
+    private static void flushRemainingBytes(final CharsetEncoder charsetEncoder,
+            final ByteBufferDestination destination, ByteBuffer temp) {
         CoderResult result;
         do {
             // write any final bytes to the output buffer once the overall input sequence has been read
@@ -160,9 +274,13 @@ public class TextEncoderHelper {
             temp = drainIfByteBufferFull(destination, temp, result);
         } while (result.isOverflow()); // byte buffer has been drained: retry
         if (!result.isUnderflow()) { // we should have fully flushed the remaining bytes
-            result.throwException();
+            throwException(result);
+        }
+        if (temp.remaining() > 0 && temp != destination.getByteBuffer()) {
+            temp.flip();
+            ByteBufferDestinationHelper.writeToUnsynchronized(temp, destination);
+            temp.clear();
         }
-        return temp;
     }
 
     /**
@@ -176,7 +294,7 @@ public class TextEncoderHelper {
         final int length = Math.min(source.length() - offset, destination.remaining());
         final char[] array = destination.array();
         final int start = destination.position();
-        source.getChars(offset, offset + length, array, start);
+        source.getChars(offset, offset + length, array, destination.arrayOffset() + start);
         destination.position(start + length);
         return length;
     }

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/MemoryMappedFileManagerTest.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/MemoryMappedFileManagerTest.java b/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/MemoryMappedFileManagerTest.java
index 6cb7e3b..84f7a4e 100644
--- a/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/MemoryMappedFileManagerTest.java
+++ b/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/MemoryMappedFileManagerTest.java
@@ -83,7 +83,7 @@ public class MemoryMappedFileManagerTest {
         final boolean immediateFlush = false;
         try (final MemoryMappedFileManager manager = MemoryMappedFileManager.getFileManager(file.getAbsolutePath(),
                 isAppend, immediateFlush, MemoryMappedFileManager.DEFAULT_REGION_LENGTH, null, null)) {
-            manager.write(new byte[initialLength], 0, initialLength);
+            manager.writeBytes(new byte[initialLength], 0, initialLength);
         }
         final int expected = initialLength * 2;
         assertEquals("appended, not overwritten", expected, file.length());

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/RandomAccessFileManagerTest.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/RandomAccessFileManagerTest.java b/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/RandomAccessFileManagerTest.java
index 3cbaa68..fac3171 100644
--- a/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/RandomAccessFileManagerTest.java
+++ b/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/RandomAccessFileManagerTest.java
@@ -39,7 +39,7 @@ public class RandomAccessFileManagerTest {
 
     /**
      * Test method for
-     * {@link org.apache.logging.log4j.core.appender.RandomAccessFileManager#write(byte[], int, int)}
+     * {@link org.apache.logging.log4j.core.appender.RandomAccessFileManager#writeBytes(byte[], int, int)}
      * .
      */
     @Test
@@ -60,7 +60,7 @@ public class RandomAccessFileManagerTest {
 
     /**
      * Test method for
-     * {@link org.apache.logging.log4j.core.appender.RandomAccessFileManager#write(byte[], int, int)}
+     * {@link org.apache.logging.log4j.core.appender.RandomAccessFileManager#writeBytes(byte[], int, int)}
      * .
      */
     @Test

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/rolling/RollingRandomAccessFileManagerTest.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/rolling/RollingRandomAccessFileManagerTest.java b/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/rolling/RollingRandomAccessFileManagerTest.java
index eb8c095..67abb55 100644
--- a/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/rolling/RollingRandomAccessFileManagerTest.java
+++ b/log4j-core/src/test/java/org/apache/logging/log4j/core/appender/rolling/RollingRandomAccessFileManagerTest.java
@@ -44,7 +44,7 @@ public class RollingRandomAccessFileManagerTest {
 
     /**
      * Test method for
-     * {@link org.apache.logging.log4j.core.appender.rolling.RollingRandomAccessFileManager#write(byte[], int, int)}
+     * {@link org.apache.logging.log4j.core.appender.rolling.RollingRandomAccessFileManager#writeBytes(byte[], int, int)}
      */
     @Test
     public void testWrite_multiplesOfBufferSize() throws IOException {
@@ -73,7 +73,7 @@ public class RollingRandomAccessFileManagerTest {
 
     /**
      * Test method for
-     * {@link org.apache.logging.log4j.core.appender.rolling.RollingRandomAccessFileManager#write(byte[], int, int)} .
+     * {@link org.apache.logging.log4j.core.appender.rolling.RollingRandomAccessFileManager#writeBytes(byte[], int, int)} .
      */
     @Test
     public void testWrite_dataExceedingBufferSize() throws IOException {

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/PatternLayoutTest.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/PatternLayoutTest.java b/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/PatternLayoutTest.java
index aab5896..4749546 100644
--- a/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/PatternLayoutTest.java
+++ b/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/PatternLayoutTest.java
@@ -88,6 +88,16 @@ public class PatternLayoutTest {
         public ByteBuffer drain(final ByteBuffer buf) {
             throw new IllegalStateException("Unexpected message larger than 2048 bytes");
         }
+
+        @Override
+        public void writeBytes(final ByteBuffer data) {
+            byteBuffer.put(data);
+        }
+
+        @Override
+        public void writeBytes(final byte[] data, final int offset, final int length) {
+            byteBuffer.put(data, offset, length);
+        }
     }
 
     private void assertToByteArray(final String expectedStr, final PatternLayout layout, final LogEvent event) {
@@ -100,7 +110,8 @@ public class PatternLayoutTest {
         layout.encode(event, destination);
         final ByteBuffer byteBuffer = destination.getByteBuffer();
         byteBuffer.flip(); // set limit to position, position back to zero
-        assertEquals(expectedStr, new String(byteBuffer.array(), 0, byteBuffer.limit()));
+        assertEquals(expectedStr, new String(byteBuffer.array(), byteBuffer.arrayOffset() + byteBuffer.position(),
+                byteBuffer.remaining()));
     }
 
     @Test
@@ -246,8 +257,8 @@ public class PatternLayoutTest {
                 .setIncludeLocation(true)
                 .setMessage(new SimpleMessage("entry")).build();
         final String result1 = new FauxLogger().formatEvent(event1, layout);
-        final String expectSuffix1 = String.format("====== PatternLayoutTest.testPatternSelector:248 entry ======%n");
-        assertTrue("Unexpected result: " + result1, result1.endsWith(expectSuffix1));
+        final String expectPattern1 = String.format(".*====== PatternLayoutTest.testPatternSelector:\\d+ entry ======%n");
+        assertTrue("Unexpected result: " + result1, result1.matches(expectPattern1));
         final LogEvent event2 = Log4jLogEvent.newBuilder() //
                 .setLoggerName(this.getClass().getName()).setLoggerFqcn("org.apache.logging.log4j.core.Logger") //
                 .setLevel(Level.INFO) //

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/SpyByteBufferDestination.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/SpyByteBufferDestination.java b/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/SpyByteBufferDestination.java
index b7e7ff8..acc8da7 100644
--- a/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/SpyByteBufferDestination.java
+++ b/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/SpyByteBufferDestination.java
@@ -60,4 +60,14 @@ public class SpyByteBufferDestination implements ByteBufferDestination {
         buf.clear();
         return buf;
     }
+
+    @Override
+    public void writeBytes(final ByteBuffer data) {
+        ByteBufferDestinationHelper.writeToUnsynchronized(data, this);
+    }
+
+    @Override
+    public void writeBytes(final byte[] data, final int offset, final int length) {
+        ByteBufferDestinationHelper.writeToUnsynchronized(data, offset, length, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/StringBuilderEncoderTest.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/StringBuilderEncoderTest.java b/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/StringBuilderEncoderTest.java
index 972d30b..51faed9 100644
--- a/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/StringBuilderEncoderTest.java
+++ b/log4j-core/src/test/java/org/apache/logging/log4j/core/layout/StringBuilderEncoderTest.java
@@ -15,12 +15,12 @@ package org.apache.logging.log4j.core.layout;/*
  * limitations under the license.
  */
 
-import org.junit.Test;
-
 import java.nio.CharBuffer;
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
 
+import org.junit.Test;
+
 import static org.junit.Assert.*;
 
 /**
@@ -181,6 +181,23 @@ public class StringBuilderEncoderTest {
     }
 
     @Test
+    public void testEncodeText_JapaneseTextUtf8DoesntFitCharBuff_DoesntFitTempByteBuff_BytesDontFitDestinationByteBuff() throws Exception {
+        final StringBuilderEncoder helper = new StringBuilderEncoder(StandardCharsets.UTF_8, 4, 5);
+        final StringBuilder text = new StringBuilder( // 日本語テスト文章日本語テスト文章
+                "\u65e5\u672c\u8a9e\u30c6\u30b9\u30c8\u6587\u7ae0\u65e5\u672c\u8a9e\u30c6\u30b9\u30c8\u6587\u7ae0");
+        final SpyByteBufferDestination destination = new SpyByteBufferDestination(3, 50);
+        helper.encode(text, destination);
+
+        assertEquals("drained", 15, destination.drainPoints.size());
+        destination.drain(destination.getByteBuffer());
+
+        final byte[] utf8 = text.toString().getBytes(StandardCharsets.UTF_8);
+        for (int i = 0; i < utf8.length; i++) {
+            assertEquals("byte at " + i, utf8[i], destination.drained.get(i));
+        }
+    }
+
+    @Test
     public void testEncodeText_JapaneseTextShiftJisDoesntFitCharBuff_BytesDontFitByteBuff() throws Exception {
         final Charset SHIFT_JIS = Charset.forName("Shift_JIS");
         final StringBuilderEncoder helper = new StringBuilderEncoder(SHIFT_JIS, 4, 8 * 1024);
@@ -189,7 +206,23 @@ public class StringBuilderEncoderTest {
         final SpyByteBufferDestination destination = new SpyByteBufferDestination(3, 50);
         helper.encode(text, destination);
 
-        assertEquals("drained", 7, destination.drainPoints.size());
+        destination.drain(destination.getByteBuffer());
+
+        final byte[] bytes = text.toString().getBytes(SHIFT_JIS);
+        for (int i = 0; i < bytes.length; i++) {
+            assertEquals("byte at " + i, bytes[i], destination.drained.get(i));
+        }
+    }
+
+    @Test
+    public void testEncodeText_JapaneseTextShiftJisDoesntFitCharBuff_DoesntFitTempByteBuff_BytesDontFitDestinationByteBuff() throws Exception {
+        final Charset SHIFT_JIS = Charset.forName("Shift_JIS");
+        final StringBuilderEncoder helper = new StringBuilderEncoder(SHIFT_JIS, 4, 5);
+        final StringBuilder text = new StringBuilder( // 日本語テスト文章日本語テスト文章
+                "\u65e5\u672c\u8a9e\u30c6\u30b9\u30c8\u6587\u7ae0\u65e5\u672c\u8a9e\u30c6\u30b9\u30c8\u6587\u7ae0");
+        final SpyByteBufferDestination destination = new SpyByteBufferDestination(3, 50);
+        helper.encode(text, destination);
+
         destination.drain(destination.getByteBuffer());
 
         final byte[] bytes = text.toString().getBytes(SHIFT_JIS);

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-core/src/test/java/org/apache/logging/log4j/test/appender/EncodingListAppender.java
----------------------------------------------------------------------
diff --git a/log4j-core/src/test/java/org/apache/logging/log4j/test/appender/EncodingListAppender.java b/log4j-core/src/test/java/org/apache/logging/log4j/test/appender/EncodingListAppender.java
index 74f8a21..4d7ee56 100644
--- a/log4j-core/src/test/java/org/apache/logging/log4j/test/appender/EncodingListAppender.java
+++ b/log4j-core/src/test/java/org/apache/logging/log4j/test/appender/EncodingListAppender.java
@@ -52,6 +52,16 @@ public class EncodingListAppender extends ListAppender {
         public ByteBuffer drain(final ByteBuffer buf) {
             throw new IllegalStateException("Unexpected message larger than 4096 bytes");
         }
+
+        @Override
+        public void writeBytes(final ByteBuffer data) {
+            byteBuffer.put(data);
+        }
+
+        @Override
+        public void writeBytes(final byte[] data, final int offset, final int length) {
+            byteBuffer.put(data, offset, length);
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-perf/src/main/java/org/apache/logging/log4j/perf/jmh/AbstractStringLayoutStringEncodingBenchmark.java
----------------------------------------------------------------------
diff --git a/log4j-perf/src/main/java/org/apache/logging/log4j/perf/jmh/AbstractStringLayoutStringEncodingBenchmark.java b/log4j-perf/src/main/java/org/apache/logging/log4j/perf/jmh/AbstractStringLayoutStringEncodingBenchmark.java
index db01a17..fa5c49f 100644
--- a/log4j-perf/src/main/java/org/apache/logging/log4j/perf/jmh/AbstractStringLayoutStringEncodingBenchmark.java
+++ b/log4j-perf/src/main/java/org/apache/logging/log4j/perf/jmh/AbstractStringLayoutStringEncodingBenchmark.java
@@ -29,6 +29,7 @@ import org.apache.logging.log4j.core.StringLayout;
 import org.apache.logging.log4j.core.impl.Log4jLogEvent;
 import org.apache.logging.log4j.core.layout.AbstractStringLayout;
 import org.apache.logging.log4j.core.layout.ByteBufferDestination;
+import org.apache.logging.log4j.core.layout.ByteBufferDestinationHelper;
 import org.apache.logging.log4j.core.layout.Encoder;
 import org.apache.logging.log4j.message.Message;
 import org.apache.logging.log4j.message.SimpleMessage;
@@ -194,9 +195,9 @@ public class AbstractStringLayoutStringEncodingBenchmark {
         return checksum;
     }
 
-    private static long consume(final byte[] bytes, final int offset, final int length) {
+    private static long consume(final byte[] bytes, final int offset, final int limit) {
         long checksum = 0;
-        for (int i = offset; i < length; i++) {
+        for (int i = offset; i < limit; i++) {
             checksum += bytes[i];
         }
         return checksum;
@@ -255,11 +256,21 @@ public class AbstractStringLayoutStringEncodingBenchmark {
         @Override
         public ByteBuffer drain(final ByteBuffer buf) {
             buf.flip();
-            consume(buf.array(), buf.position(), buf.limit());
+            consume(buf.array(), buf.arrayOffset() + buf.position(), buf.arrayOffset() + buf.limit());
             buf.clear();
             return buf;
         }
 
+        @Override
+        public void writeBytes(ByteBuffer data) {
+            ByteBufferDestinationHelper.writeToUnsynchronized(data, this);
+        }
+
+        @Override
+        public void writeBytes(final byte[] data, final int offset, final int length) {
+            ByteBufferDestinationHelper.writeToUnsynchronized(data, offset, length, this);
+        }
+
         public void reset() {
             buffer.clear();
         }

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-perf/src/main/java/org/apache/logging/log4j/perf/jmh/TextEncoderHelperBenchmark.java
----------------------------------------------------------------------
diff --git a/log4j-perf/src/main/java/org/apache/logging/log4j/perf/jmh/TextEncoderHelperBenchmark.java b/log4j-perf/src/main/java/org/apache/logging/log4j/perf/jmh/TextEncoderHelperBenchmark.java
index bf67c9d..2457061 100644
--- a/log4j-perf/src/main/java/org/apache/logging/log4j/perf/jmh/TextEncoderHelperBenchmark.java
+++ b/log4j-perf/src/main/java/org/apache/logging/log4j/perf/jmh/TextEncoderHelperBenchmark.java
@@ -28,6 +28,7 @@ import org.apache.logging.log4j.ThreadContext.ContextStack;
 import org.apache.logging.log4j.core.LogEvent;
 import org.apache.logging.log4j.core.impl.Log4jLogEvent;
 import org.apache.logging.log4j.core.layout.ByteBufferDestination;
+import org.apache.logging.log4j.core.layout.ByteBufferDestinationHelper;
 import org.apache.logging.log4j.core.layout.PatternLayout;
 import org.apache.logging.log4j.core.layout.StringBuilderEncoder;
 import org.apache.logging.log4j.message.Message;
@@ -80,6 +81,16 @@ public class TextEncoderHelperBenchmark {
             buf.clear();
             return buf;
         }
+
+        @Override
+        public void writeBytes(final ByteBuffer data) {
+            ByteBufferDestinationHelper.writeToUnsynchronized(data, this);
+        }
+
+        @Override
+        public void writeBytes(final byte[] data, final int offset, final int length) {
+            ByteBufferDestinationHelper.writeToUnsynchronized(data, offset, length, this);
+        }
     }
 
     private static LogEvent createLogEvent() {
@@ -182,7 +193,7 @@ public class TextEncoderHelperBenchmark {
         final int length = Math.min(source.length() - offset, destination.remaining());
         final char[] array = destination.array();
         final int start = destination.position();
-        source.getChars(offset, offset+length, array, start);
+        source.getChars(offset, offset + length, array, destination.arrayOffset() + start);
         destination.position(start + length);
         return length;
     }

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-perf/src/main/java/org/apache/logging/log4j/perf/nogc/DemoAppender.java
----------------------------------------------------------------------
diff --git a/log4j-perf/src/main/java/org/apache/logging/log4j/perf/nogc/DemoAppender.java b/log4j-perf/src/main/java/org/apache/logging/log4j/perf/nogc/DemoAppender.java
index df9ade0..76ef5de 100644
--- a/log4j-perf/src/main/java/org/apache/logging/log4j/perf/nogc/DemoAppender.java
+++ b/log4j-perf/src/main/java/org/apache/logging/log4j/perf/nogc/DemoAppender.java
@@ -22,6 +22,7 @@ import org.apache.logging.log4j.core.Layout;
 import org.apache.logging.log4j.core.LogEvent;
 import org.apache.logging.log4j.core.appender.AbstractAppender;
 import org.apache.logging.log4j.core.layout.ByteBufferDestination;
+import org.apache.logging.log4j.core.layout.ByteBufferDestinationHelper;
 
 /**
  * Demo Appender that does not do any I/O.
@@ -47,10 +48,10 @@ public class DemoAppender extends AbstractAppender implements ByteBufferDestinat
         }
     }
 
-    private void consume(final byte[] data, final int offset, final int length) {
+    private void consume(final byte[] data, final int offset, final int limit) {
         // need to do something with the result or the JVM may optimize everything away
         long sum = 0;
-        for (int i = offset; i < length; i++) {
+        for (int i = offset; i < limit; i++) {
             sum += data[i];
         }
         checksum += sum;
@@ -64,8 +65,18 @@ public class DemoAppender extends AbstractAppender implements ByteBufferDestinat
     @Override
     public ByteBuffer drain(final ByteBuffer buf) {
         buf.flip();
-        consume(buf.array(), buf.position(), buf.limit());
+        consume(buf.array(), buf.arrayOffset() + buf.position(), buf.arrayOffset() + buf.limit());
         buf.clear();
         return buf;
     }
+
+    @Override
+    public void writeBytes(ByteBuffer data) {
+        ByteBufferDestinationHelper.writeToUnsynchronized(data, this);
+    }
+
+    @Override
+    public void writeBytes(final byte[] data, final int offset, final int length) {
+        ByteBufferDestinationHelper.writeToUnsynchronized(data, offset, length, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/log4j-perf/src/main/java/org/apache/logging/log4j/perf/util/DemoAppender.java
----------------------------------------------------------------------
diff --git a/log4j-perf/src/main/java/org/apache/logging/log4j/perf/util/DemoAppender.java b/log4j-perf/src/main/java/org/apache/logging/log4j/perf/util/DemoAppender.java
index 30fa0d6..bbdd1b7 100644
--- a/log4j-perf/src/main/java/org/apache/logging/log4j/perf/util/DemoAppender.java
+++ b/log4j-perf/src/main/java/org/apache/logging/log4j/perf/util/DemoAppender.java
@@ -20,6 +20,7 @@ import org.apache.logging.log4j.core.Layout;
 import org.apache.logging.log4j.core.LogEvent;
 import org.apache.logging.log4j.core.appender.AbstractAppender;
 import org.apache.logging.log4j.core.layout.ByteBufferDestination;
+import org.apache.logging.log4j.core.layout.ByteBufferDestinationHelper;
 import org.apache.logging.log4j.core.util.Constants;
 
 import java.nio.ByteBuffer;
@@ -60,6 +61,16 @@ public class DemoAppender extends AbstractAppender implements ByteBufferDestinat
         return buf;
     }
 
+    @Override
+    public void writeBytes(ByteBuffer data) {
+        ByteBufferDestinationHelper.writeToUnsynchronized(data, this);
+    }
+
+    @Override
+    public void writeBytes(final byte[] data, final int offset, final int length) {
+        ByteBufferDestinationHelper.writeToUnsynchronized(data, offset, length, this);
+    }
+
     private void consume(final byte[] data, final int offset, final int length) {
         // need to do something with the result or the JVM may optimize everything away
         long sum = 0;

http://git-wip-us.apache.org/repos/asf/logging-log4j2/blob/d92cc27d/src/changes/changes.xml
----------------------------------------------------------------------
diff --git a/src/changes/changes.xml b/src/changes/changes.xml
index 36b0a54..8bdbe93 100644
--- a/src/changes/changes.xml
+++ b/src/changes/changes.xml
@@ -31,6 +31,9 @@
          - "remove" - Removed
     -->
     <release version="2.9.0" date="2017-MM-DD" description="GA Release 2.9.0">
+      <action issue="LOG4J2-1874" dev="rpopma" type="add" due-to="Roman Leventov">
+        Added methods ::writeBytes(ByteBuffer) and ::writeBytes(byte[], int, int) to ByteBufferDestination interface and use these methods in TextEncoderHelper where possible to prepare for future enhancements to reduce lock contention.
+      </action>
       <action issue="LOG4J2-1943" dev="rgoers" type="fix">
         The eventPrefix attribute was being ignored in the RFC5424Layout.
       </action>